You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/01/16 17:57:46 UTC

[1/8] incubator-nifi git commit: NIFI-273: Moved getAvailableRelationships from ProcessSession to ProcessContext

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop db2360998 -> 73384b23d


NIFI-273: Moved getAvailableRelationships from ProcessSession to ProcessContext


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

Branch: refs/heads/develop
Commit: 3a7b8de0e30e7d4dac877611d0aa9bf67bfadffd
Parents: 94a06fc
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Jan 16 11:27:22 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Jan 16 11:27:22 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/controller/StandardFunnel.java  |  4 +--
 .../repository/BatchingSessionFactory.java      |  5 ----
 .../repository/StandardProcessSession.java      |  5 ----
 .../scheduling/ConnectableProcessContext.java   | 21 +++++++++++++++
 .../nifi/processor/StandardProcessContext.java  | 28 ++++++++++++++++++++
 .../processor/StandardSchedulingContext.java    |  6 +++++
 .../protocol/socket/SocketClientProtocol.java   |  2 +-
 .../socket/SocketFlowFileServerProtocol.java    |  2 +-
 .../processors/standard/DistributeLoad.java     | 16 +++++------
 .../nifi/processors/standard/ListenHTTP.java    |  2 ++
 .../nifi/processors/standard/ListenUDP.java     |  2 +-
 .../processors/standard/PutFileTransfer.java    |  2 +-
 .../standard/servlets/ListenHTTPServlet.java    |  7 ++++-
 .../processors/standard/TestDistributeLoad.java |  1 +
 .../apache/nifi/processor/ProcessContext.java   |  8 ++++++
 .../apache/nifi/processor/ProcessSession.java   |  6 -----
 .../apache/nifi/util/MockProcessContext.java    | 23 ++++++++++++++++
 .../apache/nifi/util/MockProcessSession.java    |  4 ---
 .../apache/nifi/util/SharedSessionState.java    | 21 +--------------
 .../nifi/util/StandardProcessorTestRunner.java  |  8 +++---
 20 files changed, 114 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
index e516f20..e34e043 100644
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@ -348,7 +348,7 @@ public class StandardFunnel implements Funnel {
     private void onTrigger(final ProcessContext context, final ProcessSession session) {
         readLock.lock();
         try {
-            Set<Relationship> available = session.getAvailableRelationships();
+            Set<Relationship> available = context.getAvailableRelationships();
             int transferred = 0;
             while (!available.isEmpty()) {
                 final List<FlowFile> flowFiles = session.get(10);
@@ -359,7 +359,7 @@ public class StandardFunnel implements Funnel {
                 transferred += flowFiles.size();
                 session.transfer(flowFiles, Relationship.ANONYMOUS);
                 session.commit();
-                available = session.getAvailableRelationships();
+                available = context.getAvailableRelationships();
             }
 
             if (transferred == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index eae2550..d5dba82 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@ -98,11 +98,6 @@ public class BatchingSessionFactory implements ProcessSessionFactory {
         }
 
         @Override
-        public Set<Relationship> getAvailableRelationships() {
-            return session.getAvailableRelationships();
-        }
-
-        @Override
         public FlowFile create() {
             return session.create();
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 3d3e854..dcb461c 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -1237,11 +1237,6 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
     }
 
     @Override
-    public Set<Relationship> getAvailableRelationships() {
-        return context.getAvailableRelationships();
-    }
-
-    @Override
     public FlowFile create() {
         final Map<String, String> attrs = new HashMap<>();
         attrs.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime()));

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 8c60e4b..acb3a01 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@ -16,13 +16,18 @@
  */
 package org.apache.nifi.controller.scheduling;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.encrypt.StringEncryptor;
@@ -30,6 +35,7 @@ import org.apache.nifi.expression.AttributeValueDecorator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 
 /**
@@ -170,4 +176,19 @@ public class ConnectableProcessContext implements ProcessContext {
     public ControllerServiceLookup getControllerServiceLookup() {
         return null;
     }
+
+    @Override
+    public Set<Relationship> getAvailableRelationships() {
+        for ( final Connection connection : connectable.getConnections() ) {
+            if ( connection.getFlowFileQueue().isFull() ) {
+                return Collections.emptySet();
+            }
+        }
+        
+        final Collection<Relationship> relationships = connectable.getRelationships();
+        if ( relationships instanceof Set ) {
+            return (Set<Relationship>) relationships;
+        }
+        return new HashSet<>(connectable.getRelationships());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 93a8c6b..cd0d31c 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -16,7 +16,9 @@
  */
 package org.apache.nifi.processor;
 
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -24,6 +26,7 @@ import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.ProcessorNode;
@@ -142,4 +145,29 @@ public class StandardProcessContext implements ProcessContext, ControllerService
     public ControllerServiceLookup getControllerServiceLookup() {
         return this;
     }
+
+    @Override
+    public Set<Relationship> getAvailableRelationships() {
+        final Set<Relationship> set = new HashSet<>();
+        for (final Relationship relationship : procNode.getRelationships()) {
+            final Collection<Connection> connections = procNode.getConnections(relationship);
+            if (connections.isEmpty()) {
+                set.add(relationship);
+            } else {
+                boolean available = true;
+                for (final Connection connection : connections) {
+                    if (connection.getFlowFileQueue().isFull()) {
+                        available = false;
+                    }
+                }
+
+                if (available) {
+                    set.add(relationship);
+                }
+            }
+        }
+
+        return set;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
index 0fe08c9..318901f 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processor;
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -104,4 +105,9 @@ public class StandardSchedulingContext implements SchedulingContext {
     public ControllerServiceLookup getControllerServiceLookup() {
         return processContext.getControllerServiceLookup();
     }
+
+    @Override
+    public Set<Relationship> getAvailableRelationships() {
+        return processContext.getAvailableRelationships();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 22ec983..d4b4f61 100644
--- a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -327,7 +327,7 @@ public class SocketClientProtocol implements ClientProtocol {
         // Commit the session so that we have persisted the data
         session.commit();
         
-        if ( session.getAvailableRelationships().isEmpty() ) {
+        if ( context.getAvailableRelationships().isEmpty() ) {
             // Confirm that we received the data and the peer can now discard it but that the peer should not
             // send any more data for a bit
             logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 88b6a41..5edd4f9 100644
--- a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -482,7 +482,7 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
         // Commit the session so that we have persisted the data
         session.commit();
         
-        if ( session.getAvailableRelationships().isEmpty() ) {
+        if ( context.getAvailableRelationships().isEmpty() ) {
             // Confirm that we received the data and the peer can now discard it but that the peer should not
             // send any more data for a bit
             logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
index a755b1a..3ac55d2 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
@@ -354,14 +354,14 @@ public class DistributeLoad extends AbstractProcessor {
         }
 
         final DistributionStrategy strategy = strategyRef.get();
-        final Set<Relationship> available = session.getAvailableRelationships();
+        final Set<Relationship> available = context.getAvailableRelationships();
         final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
         final boolean allDestinationsAvailable = (available.size() == numRelationships);
         if (!allDestinationsAvailable && strategy.requiresAllDestinationsAvailable()) {
             return;
         }
 
-        final Relationship relationship = strategy.mapToRelationship(session, flowFile);
+        final Relationship relationship = strategy.mapToRelationship(context, flowFile);
         if (relationship == null) {
             // can't transfer the FlowFiles. Roll back and yield
             session.rollback();
@@ -403,7 +403,7 @@ public class DistributeLoad extends AbstractProcessor {
          * @param flowFiles
          * @return
          */
-        Relationship mapToRelationship(ProcessSession session, FlowFile flowFile);
+        Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
 
         boolean requiresAllDestinationsAvailable();
     }
@@ -413,7 +413,7 @@ public class DistributeLoad extends AbstractProcessor {
         private final AtomicLong counter = new AtomicLong(0L);
 
         @Override
-        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
             final int numRelationships = relationshipList.size();
 
@@ -427,7 +427,7 @@ public class DistributeLoad extends AbstractProcessor {
                 final long counterValue = counter.getAndIncrement();
                 final int idx = (int) (counterValue % numRelationships);
                 relationship = relationshipList.get(idx);
-                foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
+                foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
                 if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
                     return null;
                 }
@@ -448,7 +448,7 @@ public class DistributeLoad extends AbstractProcessor {
         private final AtomicLong counter = new AtomicLong(0L);
 
         @Override
-        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
             final long counterValue = counter.getAndIncrement();
             final int idx = (int) (counterValue % relationshipList.size());
@@ -467,7 +467,7 @@ public class DistributeLoad extends AbstractProcessor {
         private final AtomicLong counter = new AtomicLong(0L);
 
         @Override
-        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
             final int numRelationships = relationshipList.size();
 
@@ -481,7 +481,7 @@ public class DistributeLoad extends AbstractProcessor {
                 final long counterValue = counter.getAndIncrement();
                 final int idx = (int) (counterValue % numRelationships);
                 relationship = relationshipList.get(idx);
-                foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
+                foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
                 if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 2b0b437..b7fe97a 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@ -117,6 +117,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
     public static final String CONTEXT_ATTRIBUTE_PROCESSOR = "processor";
     public static final String CONTEXT_ATTRIBUTE_LOGGER = "logger";
     public static final String CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER = "sessionFactoryHolder";
+    public static final String CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER = "processContextHolder";
     public static final String CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN = "authorityPattern";
     public static final String CONTEXT_ATTRIBUTE_HEADER_PATTERN = "headerPattern";
     public static final String CONTEXT_ATTRIBUTE_FLOWFILE_MAP = "flowFileMap";
@@ -240,6 +241,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESSOR, this);
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_LOGGER, getLogger());
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER, sessionFactoryReference);
+        contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER, context);
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_FLOWFILE_MAP, flowFileMap);
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN, Pattern.compile(context.getProperty(AUTHORIZED_DN_PATTERN).getValue()));
         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_STREAM_THROTTLER, streamThrottler);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
index 65b3c66..43d8395 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
@@ -377,7 +377,7 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
                                     // this will throttle the processing of the received datagrams. If there are no more
                                     // buffers to read into because none have been returned to the pool via consumer.process(),
                                     // then the desired back pressure on the channel is created.
-                                    if (session.getAvailableRelationships().size() > 0) {
+                                    if (context.getAvailableRelationships().size() > 0) {
                                         consumer.process();
                                         if (flowFileCount == newFlowFiles.size()) {
                                             // no new datagrams received, need to throttle this thread back so it does

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index 31e5105..da80546 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@ -148,7 +148,7 @@ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractPr
 
                 session.transfer(flowFile, conflictResult.getRelationship());
                 session.commit();
-            } while (isScheduled() && (getRelationships().size() == session.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
+            } while (isScheduled() && (getRelationships().size() == context.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
         } catch (final IOException e) {
             context.yield();
             logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e});

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
index cae61f0..1cf5f1f 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
@@ -48,6 +48,7 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.stream.io.StreamThrottler;
 import org.apache.nifi.logging.ProcessorLog;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.io.OutputStreamCallback;
@@ -87,6 +88,7 @@ public class ListenHTTPServlet extends HttpServlet {
 
     private ProcessorLog logger;
     private AtomicReference<ProcessSessionFactory> sessionFactoryHolder;
+    private volatile ProcessContext processContext;
     private Pattern authorizedPattern;
     private Pattern headerPattern;
     private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
@@ -103,6 +105,7 @@ public class ListenHTTPServlet extends HttpServlet {
         final ServletContext context = config.getServletContext();
         this.logger = (ProcessorLog) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_LOGGER);
         this.sessionFactoryHolder = (AtomicReference<ProcessSessionFactory>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER);
+        this.processContext = (ProcessContext) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER);
         this.authorizedPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN);
         this.headerPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_HEADER_PATTERN);
         this.flowFileMap = (ConcurrentMap<String, FlowFileEntryTimeWrapper>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_FLOWFILE_MAP);
@@ -118,6 +121,8 @@ public class ListenHTTPServlet extends HttpServlet {
 
     @Override
     protected void doPost(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+        final ProcessContext context = processContext;
+        
         ProcessSessionFactory sessionFactory;
         do {
             sessionFactory = sessionFactoryHolder.get();
@@ -136,7 +141,7 @@ public class ListenHTTPServlet extends HttpServlet {
         try {
             final long n = filesReceived.getAndIncrement() % FILES_BEFORE_CHECKING_DESTINATION_SPACE;
             if (n == 0 || !spaceAvailable.get()) {
-                if (session.getAvailableRelationships().isEmpty()) {
+                if (context.getAvailableRelationships().isEmpty()) {
                     spaceAvailable.set(false);
                     if (logger.isDebugEnabled()) {
                         logger.debug("Received request from " + request.getRemoteHost() + " but no space available; Indicating Service Unavailable");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java b/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
index a6402e4..ab4c978 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
@@ -132,6 +132,7 @@ public class TestDistributeLoad {
         testRunner.assertQueueEmpty();
 
         for (int i = 1; i <= 100; i++) {
+            System.out.println(i);
             testRunner.assertTransferCount(String.valueOf(i), (i == 50) ? 0 : 1);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
index 9e04439..7fa183f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processor;
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -121,4 +122,11 @@ public interface ProcessContext {
      * @return
      */
     ControllerServiceLookup getControllerServiceLookup();
+    
+    /**
+     * @return the set of all relationships for which space is available to
+     * receive new objects
+     */
+    Set<Relationship> getAvailableRelationships();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
index 09d1bd2..d3de916 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
@@ -166,12 +166,6 @@ public interface ProcessSession {
     QueueSize getQueueSize();
 
     /**
-     * @return the set of all relationships for which space is available to
-     * receive new objects
-     */
-    Set<Relationship> getAvailableRelationships();
-
-    /**
      * Creates a new FlowFile in the repository with no content and without any
      * linkage to a parent FlowFile. This method is appropriate only when data
      * is received or created from an external system. Otherwise, this method

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index 6e5f65d..15591d7 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@ -21,10 +21,12 @@ import static java.util.Objects.requireNonNull;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -32,6 +34,8 @@ import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.SchedulingContext;
 import org.junit.Assert;
 
@@ -45,6 +49,8 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
     private boolean enableExpressionValidation = false;
     private boolean allowExpressionValidation = true;
 
+    private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
+
     /**
      * Creates a new MockProcessContext for the given Processor
      *
@@ -258,4 +264,21 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
     public void leaseControllerService(final String identifier) {
     }
 
+    public Set<Relationship> getAvailableRelationships() {
+        if ( !(component instanceof Processor) ) {
+            return Collections.emptySet();
+        }
+        
+        final Set<Relationship> relationships = new HashSet<>(((Processor) component).getRelationships());
+        relationships.removeAll(unavailableRelationships);
+        return relationships;
+    }
+
+    public void setUnavailableRelationships(final Set<Relationship> relationships) {
+        this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
+    }
+
+    public Set<Relationship> getUnavailableRelationships() {
+        return unavailableRelationships;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index 552780c..ea55b34 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@ -329,10 +329,6 @@ public class MockProcessSession implements ProcessSession {
         return newFlowFile;
     }
 
-    @Override
-    public Set<Relationship> getAvailableRelationships() {
-        return sharedState.getAvailableRelationships();
-    }
 
     @Override
     public MockFlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java b/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
index 96bef71..13a87de 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
@@ -16,49 +16,30 @@
  */
 package org.apache.nifi.util;
 
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.ProvenanceReporter;
 
 public class SharedSessionState {
 
     private final MockFlowFileQueue flowFileQueue;
     private final ProvenanceReporter provenanceReporter;
+    @SuppressWarnings("unused")
     private final Processor processor;
     private final AtomicLong flowFileIdGenerator;
     private final ConcurrentMap<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
 
-    private volatile Set<Relationship> unavailableRelationships;
 
     public SharedSessionState(final Processor processor, final AtomicLong flowFileIdGenerator) {
         flowFileQueue = new MockFlowFileQueue();
         provenanceReporter = new MockProvenanceReporter();
-        unavailableRelationships = new HashSet<>();
         this.flowFileIdGenerator = flowFileIdGenerator;
         this.processor = processor;
     }
 
-    public Set<Relationship> getAvailableRelationships() {
-        final Set<Relationship> relationships = new HashSet<>(processor.getRelationships());
-        relationships.removeAll(unavailableRelationships);
-        return relationships;
-    }
-
-    public void setUnavailableRelationships(final Set<Relationship> relationships) {
-        this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
-    }
-
-    public Set<Relationship> getUnavailableRelationships() {
-        return unavailableRelationships;
-    }
-
     public MockFlowFileQueue getFlowFileQueue() {
         return flowFileQueue;
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 54b611d..40d5035 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -426,9 +426,9 @@ public class StandardProcessorTestRunner implements TestRunner {
 
     @Override
     public void setRelationshipAvailable(final Relationship relationship) {
-        final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
+        final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
         unavailable.remove(relationship);
-        sharedState.setUnavailableRelationships(unavailable);
+        context.setUnavailableRelationships(unavailable);
     }
 
     @Override
@@ -438,9 +438,9 @@ public class StandardProcessorTestRunner implements TestRunner {
 
     @Override
     public void setRelationshipUnavailable(final Relationship relationship) {
-        final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
+        final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
         unavailable.add(relationship);
-        sharedState.setUnavailableRelationships(unavailable);
+        context.setUnavailableRelationships(unavailable);
     }
 
     @Override


[5/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 0000000,8c60e4b..acb3a01
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@@ -1,0 -1,173 +1,194 @@@
+ /*
+  * 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.controller.scheduling;
+ 
++import java.util.Collection;
++import java.util.Collections;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.Map;
++import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.connectable.Connectable;
++import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.encrypt.StringEncryptor;
+ import org.apache.nifi.expression.AttributeValueDecorator;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
++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
+  */
+ public class ConnectableProcessContext implements ProcessContext {
+ 
+     private final Connectable connectable;
+     private final StringEncryptor encryptor;
+ 
+     public ConnectableProcessContext(final Connectable connectable, final StringEncryptor encryptor) {
+         this.connectable = connectable;
+         this.encryptor = encryptor;
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+         return getProperty(descriptor.getName());
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final String propertyName) {
+         return new PropertyValue() {
+             @Override
+             public String getValue() {
+                 return null;
+             }
+ 
+             @Override
+             public Integer asInteger() {
+                 return null;
+             }
+ 
+             @Override
+             public Long asLong() {
+                 return null;
+             }
+ 
+             @Override
+             public Boolean asBoolean() {
+                 return null;
+             }
+ 
+             @Override
+             public Float asFloat() {
+                 return null;
+             }
+ 
+             @Override
+             public Double asDouble() {
+                 return null;
+             }
+ 
+             @Override
+             public Long asTimePeriod(final TimeUnit timeUnit) {
+                 return null;
+             }
+ 
+             @Override
+             public Double asDataSize(final DataUnit dataUnit) {
+                 return null;
+             }
+ 
+             @Override
+             public PropertyValue evaluateAttributeExpressions() throws ProcessException {
+                 return this;
+             }
+ 
+             @Override
+             public PropertyValue evaluateAttributeExpressions(final FlowFile flowFile) throws ProcessException {
+                 return this;
+             }
+ 
+             @Override
+             public PropertyValue evaluateAttributeExpressions(final AttributeValueDecorator decorator) throws ProcessException {
+                 return this;
+             }
+ 
+             @Override
+             public PropertyValue evaluateAttributeExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
+                 return this;
+             }
+ 
+             @Override
+             public ControllerService asControllerService() {
+                 return null;
+             }
+ 
+             @Override
+             public <T extends ControllerService> T asControllerService(Class<T> serviceType) throws IllegalArgumentException {
+                 return null;
+             }
+ 
+             @Override
+             public boolean isSet() {
+                 return false;
+             }
+         };
+     }
+ 
+     @Override
+     public PropertyValue newPropertyValue(String rawValue) {
+         throw new UnsupportedOperationException();
+     }
+ 
+     @Override
+     public void yield() {
+         connectable.yield();
+     }
+ 
+     @Override
+     public int getMaxConcurrentTasks() {
+         return connectable.getMaxConcurrentTasks();
+     }
+ 
+     @Override
+     public String getAnnotationData() {
+         return null;
+     }
+ 
+     @Override
+     public Map<PropertyDescriptor, String> getProperties() {
+         return new HashMap<>();
+     }
+ 
+     @Override
+     public String decrypt(String encrypted) {
+         return encryptor.decrypt(encrypted);
+     }
+ 
+     @Override
+     public String encrypt(String unencrypted) {
+         return encryptor.encrypt(unencrypted);
+     }
+ 
+     @Override
+     public ControllerServiceLookup getControllerServiceLookup() {
+         return null;
+     }
++
++    @Override
++    public Set<Relationship> getAvailableRelationships() {
++        for ( final Connection connection : connectable.getConnections() ) {
++            if ( connection.getFlowFileQueue().isFull() ) {
++                return Collections.emptySet();
++            }
++        }
++        
++        final Collection<Relationship> relationships = connectable.getRelationships();
++        if ( relationships instanceof Set ) {
++            return (Set<Relationship>) relationships;
++        }
++        return new HashSet<>(connectable.getRelationships());
++    }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 0000000,93a8c6b..cd0d31c
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@@ -1,0 -1,145 +1,173 @@@
+ /*
+  * 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.processor;
+ 
++import java.util.Collection;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ 
+ import org.apache.nifi.attribute.expression.language.PreparedQuery;
+ import org.apache.nifi.attribute.expression.language.Query;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
++import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.service.ControllerServiceProvider;
+ import org.apache.nifi.encrypt.StringEncryptor;
+ 
+ public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
+ 
+     private final ProcessorNode procNode;
+     private final ControllerServiceProvider controllerServiceProvider;
+     private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
+     private final StringEncryptor encryptor;
+ 
+     public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor) {
+         this.procNode = processorNode;
+         this.controllerServiceProvider = controllerServiceProvider;
+         this.encryptor = encryptor;
+ 
+         preparedQueries = new HashMap<>();
+         for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
+             final PropertyDescriptor desc = entry.getKey();
+             String value = entry.getValue();
+             if (value == null) {
+                 value = desc.getDefaultValue();
+             }
+ 
+             final PreparedQuery pq = Query.prepare(value);
+             preparedQueries.put(desc, pq);
+         }
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+         return getProperty(descriptor.getName());
+     }
+ 
+     /**
+      * <p>
+      * Returns the currently configured value for the property with the given
+      * name.
+      * </p>
+      */
+     @Override
+     public PropertyValue getProperty(final String propertyName) {
+         final Processor processor = procNode.getProcessor();
+         final PropertyDescriptor descriptor = processor.getPropertyDescriptor(propertyName);
+         if (descriptor == null) {
+             return null;
+         }
+ 
+         final String setPropertyValue = procNode.getProperty(descriptor);
+         final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
+ 
+         return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor));
+     }
+ 
+     @Override
+     public PropertyValue newPropertyValue(final String rawValue) {
+         return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue));
+     }
+ 
+     @Override
+     public void yield() {
+         procNode.yield();
+     }
+ 
+     @Override
+     public ControllerService getControllerService(final String serviceIdentifier) {
+         return controllerServiceProvider.getControllerService(serviceIdentifier);
+     }
+ 
+     @Override
+     public int getMaxConcurrentTasks() {
+         return procNode.getMaxConcurrentTasks();
+     }
+ 
+     @Override
+     public String getAnnotationData() {
+         return procNode.getAnnotationData();
+     }
+ 
+     @Override
+     public Map<PropertyDescriptor, String> getProperties() {
+         return procNode.getProperties();
+     }
+ 
+     @Override
+     public String encrypt(final String unencrypted) {
+         return encryptor.encrypt(unencrypted);
+     }
+ 
+     @Override
+     public String decrypt(final String encrypted) {
+         return encryptor.decrypt(encrypted);
+     }
+ 
+     @Override
+     public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
+         if (!serviceType.isInterface()) {
+             throw new IllegalArgumentException("ControllerServices may be referenced only via their interfaces; " + serviceType + " is not an interface");
+         }
+         return controllerServiceProvider.getControllerServiceIdentifiers(serviceType);
+     }
+ 
+     @Override
+     public boolean isControllerServiceEnabled(final ControllerService service) {
+         return controllerServiceProvider.isControllerServiceEnabled(service);
+     }
+ 
+     @Override
+     public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+         return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier);
+     }
+ 
+     @Override
+     public ControllerServiceLookup getControllerServiceLookup() {
+         return this;
+     }
++
++    @Override
++    public Set<Relationship> getAvailableRelationships() {
++        final Set<Relationship> set = new HashSet<>();
++        for (final Relationship relationship : procNode.getRelationships()) {
++            final Collection<Connection> connections = procNode.getConnections(relationship);
++            if (connections.isEmpty()) {
++                set.add(relationship);
++            } else {
++                boolean available = true;
++                for (final Connection connection : connections) {
++                    if (connection.getFlowFileQueue().isFull()) {
++                        available = false;
++                    }
++                }
++
++                if (available) {
++                    set.add(relationship);
++                }
++            }
++        }
++
++        return set;
++    }
++    
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
index 0000000,0fe08c9..318901f
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
@@@ -1,0 -1,107 +1,113 @@@
+ /*
+  * 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.processor;
+ 
+ import java.util.Map;
++import java.util.Set;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.service.ControllerServiceNode;
+ import org.apache.nifi.controller.service.ControllerServiceProvider;
+ 
+ public class StandardSchedulingContext implements SchedulingContext {
+ 
+     private final ProcessContext processContext;
+     private final ControllerServiceProvider serviceProvider;
+     private final ProcessorNode processorNode;
+ 
+     public StandardSchedulingContext(final ProcessContext processContext, final ControllerServiceProvider serviceProvider, final ProcessorNode processorNode) {
+         this.processContext = processContext;
+         this.serviceProvider = serviceProvider;
+         this.processorNode = processorNode;
+     }
+ 
+     @Override
+     public void leaseControllerService(final String identifier) {
+         final ControllerServiceNode serviceNode = serviceProvider.getControllerServiceNode(identifier);
+         if (serviceNode == null) {
+             throw new IllegalArgumentException("Cannot lease Controller Service because no Controller Service exists with identifier " + identifier);
+         }
+ 
+         if (serviceNode.isDisabled()) {
+             throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getControllerService() + " is currently disabled");
+         }
+ 
+         if (!serviceNode.isValid()) {
+             throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getControllerService() + " is not currently valid");
+         }
+ 
+         serviceNode.addReference(processorNode);
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+         return processContext.getProperty(descriptor);
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final String propertyName) {
+         return processContext.getProperty(propertyName);
+     }
+ 
+     @Override
+     public PropertyValue newPropertyValue(final String rawValue) {
+         return processContext.newPropertyValue(rawValue);
+     }
+ 
+     @Override
+     public void yield() {
+         processContext.yield();
+     }
+ 
+     @Override
+     public int getMaxConcurrentTasks() {
+         return processContext.getMaxConcurrentTasks();
+     }
+ 
+     @Override
+     public String getAnnotationData() {
+         return processContext.getAnnotationData();
+     }
+ 
+     @Override
+     public Map<PropertyDescriptor, String> getProperties() {
+         return processContext.getProperties();
+     }
+ 
+     @Override
+     public String encrypt(final String unencrypted) {
+         return processContext.encrypt(unencrypted);
+     }
+ 
+     @Override
+     public String decrypt(final String encrypted) {
+         return processContext.decrypt(encrypted);
+     }
+ 
+     @Override
+     public ControllerServiceLookup getControllerServiceLookup() {
+         return processContext.getControllerServiceLookup();
+     }
++
++    @Override
++    public Set<Relationship> getAvailableRelationships() {
++        return processContext.getAvailableRelationships();
++    }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 0000000,22ec983..d4b4f61
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@@ -1,0 -1,510 +1,510 @@@
+ /*
+  * 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.remote.protocol.socket;
+ 
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.zip.CRC32;
+ import java.util.zip.CheckedInputStream;
+ import java.util.zip.CheckedOutputStream;
+ 
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.remote.Peer;
+ import org.apache.nifi.remote.PeerStatus;
+ import org.apache.nifi.remote.RemoteGroupPort;
+ import org.apache.nifi.remote.RemoteResourceFactory;
+ import org.apache.nifi.remote.StandardVersionNegotiator;
+ import org.apache.nifi.remote.VersionNegotiator;
+ import org.apache.nifi.remote.codec.FlowFileCodec;
+ import org.apache.nifi.remote.codec.StandardFlowFileCodec;
+ import org.apache.nifi.remote.exception.HandshakeException;
+ import org.apache.nifi.remote.exception.ProtocolException;
+ import org.apache.nifi.remote.io.CompressionInputStream;
+ import org.apache.nifi.remote.io.CompressionOutputStream;
+ import org.apache.nifi.remote.protocol.ClientProtocol;
+ import org.apache.nifi.remote.protocol.CommunicationsSession;
+ import org.apache.nifi.remote.protocol.RequestType;
+ import org.apache.nifi.util.FormatUtils;
+ import org.apache.nifi.util.StopWatch;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ public class SocketClientProtocol implements ClientProtocol {
+     private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+ 
+     
+     private RemoteGroupPort port;
+     private boolean useCompression;
+     
+     private String commsIdentifier;
+     private boolean handshakeComplete = false;
+     
+     private final Logger logger = LoggerFactory.getLogger(SocketClientProtocol.class);
+     
+     private Response handshakeResponse = null;
+     private boolean readyForFileTransfer = false;
+     private String transitUriPrefix = null;
+     
+     private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+     
+     public SocketClientProtocol() {
+     }
+ 
+     public void setPort(final RemoteGroupPort port) {
+         this.port = port;
+         this.useCompression = port.isUseCompression();
+     }
+     
+     @Override
+     public void handshake(final Peer peer) throws IOException, HandshakeException {
+         if ( handshakeComplete ) {
+             throw new IllegalStateException("Handshake has already been completed");
+         }
+         commsIdentifier = UUID.randomUUID().toString();
+         logger.debug("{} handshaking with {}", this, peer);
+         
+         final Map<HandshakeProperty, String> properties = new HashMap<>();
+         properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
+         properties.put(HandshakeProperty.PORT_IDENTIFIER, port.getIdentifier());
+         properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(
+             port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS)) );
+         
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         commsSession.setTimeout(port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS));
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         
+         dos.writeUTF(commsIdentifier);
+         
+         if ( versionNegotiator.getVersion() >= 3 ) {
+             dos.writeUTF(peer.getUrl());
+             transitUriPrefix = peer.getUrl();
+             
+             if ( !transitUriPrefix.endsWith("/") ) {
+                 transitUriPrefix = transitUriPrefix + "/";
+             }
+         }
+         
+         dos.writeInt(properties.size());
+         for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
+             dos.writeUTF(entry.getKey().name());
+             dos.writeUTF(entry.getValue());
+         }
+         
+         dos.flush();
+         
+         try {
+             handshakeResponse = Response.read(dis);
+         } catch (final ProtocolException e) {
+             throw new HandshakeException(e);
+         }
+         
+         switch (handshakeResponse.getCode()) {
+             case PORT_NOT_IN_VALID_STATE:
+             case UNKNOWN_PORT:
+             case PORTS_DESTINATION_FULL:
+                 break;
+             case PROPERTIES_OK:
+                 readyForFileTransfer = true;
+                 break;
+             default:
+                 logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[] {
+                     this, handshakeResponse, peer});
+                 peer.close();
+                 throw new HandshakeException("Received unexpected response " + handshakeResponse);
+         }
+         
+         logger.debug("{} Finished handshake with {}", this, peer);
+         handshakeComplete = true;
+     }
+     
+     public boolean isReadyForFileTransfer() {
+         return readyForFileTransfer;
+     }
+     
+     public boolean isPortInvalid() {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not completed successfully");
+         }
+         return handshakeResponse.getCode() == ResponseCode.PORT_NOT_IN_VALID_STATE;
+     }
+     
+     public boolean isPortUnknown() {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not completed successfully");
+         }
+         return handshakeResponse.getCode() == ResponseCode.UNKNOWN_PORT;
+     }
+     
+     public boolean isDestinationFull() {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not completed successfully");
+         }
+         return handshakeResponse.getCode() == ResponseCode.PORTS_DESTINATION_FULL;
+     }
+     
+     @Override
+     public Set<PeerStatus> getPeerStatuses(final Peer peer) throws IOException {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not been performed");
+         }
+         
+         logger.debug("{} Get Peer Statuses from {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         
+         RequestType.REQUEST_PEER_LIST.writeRequestType(dos);
+         dos.flush();
+         final int numPeers = dis.readInt();
+         final Set<PeerStatus> peers = new HashSet<>(numPeers);
+         for (int i=0; i < numPeers; i++) {
+             final String hostname = dis.readUTF();
+             final int port = dis.readInt();
+             final boolean secure = dis.readBoolean();
+             final int flowFileCount = dis.readInt();
+             peers.add(new PeerStatus(hostname, port, secure, flowFileCount));
+         }
+         
+         logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);
+         return peers;
+     }
+     
+     @Override
+     public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not been performed");
+         }
+ 
+         logger.debug("{} Negotiating Codec with {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ 
+         RequestType.NEGOTIATE_FLOWFILE_CODEC.writeRequestType(dos);
+         
+         FlowFileCodec codec = new StandardFlowFileCodec();
+         try {
+             codec = (FlowFileCodec) RemoteResourceFactory.initiateResourceNegotiation(codec, dis, dos);
+         } catch (HandshakeException e) {
+             throw new ProtocolException(e.toString());
+         }
+         logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[] {this, codec, commsSession});
+ 
+         return codec;
+     }
+ 
+     
+     @Override
+     public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not been performed");
+         }
+         if ( !readyForFileTransfer ) {
+             throw new IllegalStateException("Cannot receive files; handshake resolution was " + handshakeResponse);
+         }
+ 
+         logger.debug("{} Receiving FlowFiles from {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         String userDn = commsSession.getUserDn();
+         if ( userDn == null ) {
+             userDn = "none";
+         }
+         
+         // Indicate that we would like to have some data
+         RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
+         dos.flush();
+         
+         // Determine if Peer will send us data or has no data to send us
+         final Response dataAvailableCode = Response.read(dis);
+         switch (dataAvailableCode.getCode()) {
+             case MORE_DATA:
+                 logger.debug("{} {} Indicates that data is available", this, peer);
+                 break;
+             case NO_MORE_DATA:
+                 logger.debug("{} No data available from {}", peer);
+                 return;
+             default:
+                 throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+         }
+ 
+         final StopWatch stopWatch = new StopWatch(true);
+         final Set<FlowFile> flowFilesReceived = new HashSet<>();
+         long bytesReceived = 0L;
+         final CRC32 crc = new CRC32();
+         
+         // Peer has data. Decode the bytes into FlowFiles until peer says he's finished sending data.
+         boolean continueTransaction = true;
+         String calculatedCRC = "";
+         while (continueTransaction) {
+             final InputStream flowFileInputStream = useCompression ? new CompressionInputStream(dis) : dis;
+             final CheckedInputStream checkedIn = new CheckedInputStream(flowFileInputStream, crc);
+             
+             final long startNanos = System.nanoTime();
+             FlowFile flowFile = codec.decode(checkedIn, session);
+             final long transmissionNanos = System.nanoTime() - startNanos;
+             final long transmissionMillis = TimeUnit.MILLISECONDS.convert(transmissionNanos, TimeUnit.NANOSECONDS);
+             
+             final String sourceFlowFileIdentifier = flowFile.getAttribute(CoreAttributes.UUID.key());
+             flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+             
+             final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
+             session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transmissionMillis);
+             
+             session.transfer(flowFile, Relationship.ANONYMOUS);
+             bytesReceived += flowFile.getSize();
+             flowFilesReceived.add(flowFile);
+             logger.debug("{} Received {} from {}", this, flowFile, peer);
+             
+             final Response transactionCode = Response.read(dis);
+             switch (transactionCode.getCode()) {
+                 case CONTINUE_TRANSACTION:
+                     logger.trace("{} Received ContinueTransaction indicator from {}", this, peer);
+                     break;
+                 case FINISH_TRANSACTION:
+                     logger.trace("{} Received FinishTransaction indicator from {}", this, peer);
+                     continueTransaction = false;
+                     calculatedCRC = String.valueOf(checkedIn.getChecksum().getValue());
+                     break;
+                 default:
+                     throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionCode);
+             }
+         }
+         
+         // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+         // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+         // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+         // session and then when we send the response back to the peer, the peer may have timed out and may not
+         // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+         // Critical Section involved in this transaction so that rather than the Critical Section being the
+         // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+         logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
+         ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+         
+         final Response confirmTransactionResponse = Response.read(dis);
+         logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
+         
+         switch (confirmTransactionResponse.getCode()) {
+             case CONFIRM_TRANSACTION:
+                 break;
+             case BAD_CHECKSUM:
+                 session.rollback();
+                 throw new IOException(this + " Received a BadChecksum response from peer " + peer);
+             default:
+                 throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
+         }
+         
+         // Commit the session so that we have persisted the data
+         session.commit();
+         
 -        if ( session.getAvailableRelationships().isEmpty() ) {
++        if ( context.getAvailableRelationships().isEmpty() ) {
+             // Confirm that we received the data and the peer can now discard it but that the peer should not
+             // send any more data for a bit
+             logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+             ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
+         } else {
+             // Confirm that we received the data and the peer can now discard it
+             logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
+             ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
+         }
+         
+         stopWatch.stop();
+         final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
+         final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
+         final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+         final String dataSize = FormatUtils.formatDataSize(bytesReceived);
+         logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
+             this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+     }
+ 
+     @Override
+     public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+         if ( !handshakeComplete ) {
+             throw new IllegalStateException("Handshake has not been performed");
+         }
+         if ( !readyForFileTransfer ) {
+             throw new IllegalStateException("Cannot transfer files; handshake resolution was " + handshakeResponse);
+         }
+ 
+         FlowFile flowFile = session.get();
+         if ( flowFile == null ) {
+             return;
+         }
+ 
+         logger.debug("{} Sending FlowFiles to {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         String userDn = commsSession.getUserDn();
+         if ( userDn == null ) {
+             userDn = "none";
+         }
+         
+         // Indicate that we would like to have some data
+         RequestType.SEND_FLOWFILES.writeRequestType(dos);
+         dos.flush();
+         
+         final StopWatch stopWatch = new StopWatch(true);
+         final CRC32 crc = new CRC32();
+         
+         long bytesSent = 0L;
+         final Set<FlowFile> flowFilesSent = new HashSet<>();
+         boolean continueTransaction = true;
+         String calculatedCRC = "";
+         final long startSendingNanos = System.nanoTime();
+         while (continueTransaction) {
+             final OutputStream flowFileOutputStream = useCompression ? new CompressionOutputStream(dos) : dos;
+             logger.debug("{} Sending {} to {}", this, flowFile, peer);
+             
+             final CheckedOutputStream checkedOutStream = new CheckedOutputStream(flowFileOutputStream, crc);
+             
+             final long startNanos = System.nanoTime();
+             flowFile = codec.encode(flowFile, session, checkedOutStream);
+             final long transferNanos = System.nanoTime() - startNanos;
+             final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+             
+             // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
+             // Otherwise, do NOT close it because we don't want to close the underlying stream
+             // (CompressionOutputStream will not close the underlying stream when it's closed)
+             if ( useCompression ) {
+                 checkedOutStream.close();
+             }
+             
+             flowFilesSent.add(flowFile);
+             bytesSent += flowFile.getSize();
+             logger.debug("{} Sent {} to {}", this, flowFile, peer);
+             
+             final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
+             session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transferMillis, false);
+             session.remove(flowFile);
+             
+             final long sendingNanos = System.nanoTime() - startSendingNanos;
+             if ( sendingNanos < BATCH_SEND_NANOS ) { 
+                 flowFile = session.get();
+             } else {
+                 flowFile = null;
+             }
+             
+             continueTransaction = (flowFile != null);
+             if ( continueTransaction ) {
+                 logger.debug("{} Sent CONTINUE_TRANSACTION indicator to {}", this, peer);
+                 ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+             } else {
+                 logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
+                 ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+                 
+                 calculatedCRC = String.valueOf( checkedOutStream.getChecksum().getValue() );
+             }
+         }
+         
+         // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+         final Response transactionConfirmationResponse = Response.read(dis);
+         if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
+             // Confirm checksum and echo back the confirmation.
+             logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
+             final String receivedCRC = transactionConfirmationResponse.getMessage();
+             
+             if ( versionNegotiator.getVersion() > 3 ) {
+                 if ( !receivedCRC.equals(calculatedCRC) ) {
+                     ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+                     session.rollback();
+                     throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
+                 }
+             }
+             
+             ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+         } else {
+             throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+         }
+ 
+         final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+ 
+         final Response transactionResponse;
+         try {
+             transactionResponse = Response.read(dis);
+         } catch (final IOException e) {
+             logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
+                     " It is unknown whether or not the peer successfully received/processed the data." +
+                     " Therefore, {} will be rolled back, possibly resulting in data duplication of {}", 
+                     this, peer, session, flowFileDescription);
+             session.rollback();
+             throw e;
+         }
+         
+         logger.debug("{} Received {} from {}", this, transactionResponse, peer);
+         if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+             peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
+         } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+             throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+         }
+         
+         // consume input stream entirely, ignoring its contents. If we
+         // don't do this, the Connection will not be returned to the pool
+         stopWatch.stop();
+         final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
+         final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+         final String dataSize = FormatUtils.formatDataSize(bytesSent);
+         
+         session.commit();
+         
+         logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+             this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+     }
+ 
+     @Override
+     public VersionNegotiator getVersionNegotiator() {
+         return versionNegotiator;
+     }
+     
+     @Override
+     public void shutdown(final Peer peer) throws IOException {
+         readyForFileTransfer = false;
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         
+         logger.debug("{} Shutting down with {}", this, peer);
+         // Indicate that we would like to have some data
+         RequestType.SHUTDOWN.writeRequestType(dos);
+         dos.flush();
+     }
+ 
+     @Override
+     public String getResourceName() {
+         return "SocketFlowFileProtocol";
+     }
+     
+     @Override
+     public String toString() {
+         return "SocketClientProtocol[CommsID=" + commsIdentifier + "]";
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 0000000,88b6a41..5edd4f9
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@@ -1,0 -1,581 +1,581 @@@
+ /*
+  * 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.remote.protocol.socket;
+ 
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.net.InetAddress;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.zip.CRC32;
+ import java.util.zip.CheckedInputStream;
+ import java.util.zip.CheckedOutputStream;
+ 
+ import org.apache.nifi.cluster.NodeInformant;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.connectable.Port;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.groups.ProcessGroup;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.remote.Peer;
+ import org.apache.nifi.remote.PortAuthorizationResult;
+ import org.apache.nifi.remote.RemoteResourceFactory;
+ import org.apache.nifi.remote.RootGroupPort;
+ import org.apache.nifi.remote.StandardVersionNegotiator;
+ import org.apache.nifi.remote.VersionNegotiator;
+ import org.apache.nifi.remote.codec.FlowFileCodec;
+ import org.apache.nifi.remote.exception.HandshakeException;
+ import org.apache.nifi.remote.exception.ProtocolException;
+ import org.apache.nifi.remote.io.CompressionInputStream;
+ import org.apache.nifi.remote.io.CompressionOutputStream;
+ import org.apache.nifi.remote.protocol.CommunicationsSession;
+ import org.apache.nifi.remote.protocol.RequestType;
+ import org.apache.nifi.remote.protocol.ServerProtocol;
+ import org.apache.nifi.util.FormatUtils;
+ import org.apache.nifi.util.NiFiProperties;
+ import org.apache.nifi.util.StopWatch;
+ 
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ public class SocketFlowFileServerProtocol implements ServerProtocol {
+     public static final String RESOURCE_NAME = "SocketFlowFileProtocol";
+     
+     private ProcessGroup rootGroup;
+     private String commsIdentifier;
+     private boolean handshakeCompleted;
+     
+     private Boolean useGzip;
+     private long requestExpirationMillis;
+     private RootGroupPort port;
+     private boolean shutdown = false;
+     private FlowFileCodec negotiatedFlowFileCodec = null;
+     private String transitUriPrefix = null;
+     
+     private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+     private final Logger logger = LoggerFactory.getLogger(SocketFlowFileServerProtocol.class);
+     
+     private static final long BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+ 
+     
+     @Override
+     public void setRootProcessGroup(final ProcessGroup group) {
+         if ( !group.isRootGroup() ) {
+             throw new IllegalArgumentException();
+         }
+         this.rootGroup = group;
+     }
+     
+     @Override
+     public void handshake(final Peer peer) throws IOException, HandshakeException {
+         if ( handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has already been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} Handshaking with {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         
+         commsIdentifier = dis.readUTF();
+         
+         if ( versionNegotiator.getVersion() >= 3 ) {
+             transitUriPrefix = dis.readUTF();
+             if ( !transitUriPrefix.endsWith("/") ) {
+                 transitUriPrefix = transitUriPrefix + "/";
+             }
+         }
+         
+         final Map<String, String> properties = new HashMap<>();
+         final int numProperties = dis.readInt();
+         for (int i=0; i < numProperties; i++) {
+             final String propertyName = dis.readUTF();
+             final String propertyValue = dis.readUTF();
+             properties.put(propertyName, propertyValue);
+         }
+         
+         // evaluate the properties received
+         boolean responseWritten = false;
+         for ( final Map.Entry<String, String> entry : properties.entrySet() ) {
+             final String propertyName = entry.getKey();
+             final String value = entry.getValue();
+             
+             final HandshakeProperty property;
+             try {
+                 property = HandshakeProperty.valueOf(propertyName);
+             } catch (final Exception e) {
+                 ResponseCode.UNKNOWN_PROPERTY_NAME.writeResponse(dos, "Unknown Property Name: " + propertyName);
+                 throw new HandshakeException("Received unknown property: " + propertyName);
+             }
+             
+             switch (property) {
+                 case GZIP: {
+                     useGzip = Boolean.parseBoolean(value);
+                     break;
+                 }
+                 case REQUEST_EXPIRATION_MILLIS:
+                     requestExpirationMillis = Long.parseLong(value);
+                     break;
+                 case PORT_IDENTIFIER: {
+                     Port receivedPort = rootGroup.getInputPort(value);
+                     if ( receivedPort == null ) {
+                         receivedPort = rootGroup.getOutputPort(value);
+                     }
+                     if ( receivedPort == null ) {
+                         logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+                         ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+                         throw new HandshakeException("Received unknown port identifier: " + value);
+                     }
+                     if ( !(receivedPort instanceof RootGroupPort) ) {
+                         logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+                         ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+                         throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort");
+                     }
+                     
+                     this.port = (RootGroupPort) receivedPort;
+                     final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
+                     if ( !portAuthResult.isAuthorized() ) {
+                         logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
+                         ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation());
+                         responseWritten = true;
+                         break;
+                     }
+                     
+                     if ( !receivedPort.isValid() ) {
+                         logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+                         ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid");
+                         responseWritten = true;
+                         break;
+                     }
+                     
+                     if ( !receivedPort.isRunning() ) {
+                         logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+                         ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running");
+                         responseWritten = true;
+                         break;
+                     }
+                     
+                     // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
+                     // we we will simply not service the request but the sender will timeout
+                     if ( getVersionNegotiator().getVersion() > 1 ) {
+                         for ( final Connection connection : port.getConnections() ) {
+                             if ( connection.getFlowFileQueue().isFull() ) {
+                                 logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort);
+                                 ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos);
+                                 responseWritten = true;
+                                 break;
+                             }
+                         }
+                     }
+                     
+                     break;
+                 }
+             }
+         }
+         
+         if ( useGzip == null ) {
+             logger.debug("Responding with ResponseCode MISSING_PROPERTY because GZIP Property missing");
+             ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.GZIP.name());
+             throw new HandshakeException("Missing Property " + HandshakeProperty.GZIP.name());
+         }
+         if ( port == null ) {
+             logger.debug("Responding with ResponseCode MISSING_PROPERTY because Port Identifier property is missing");
+             ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.PORT_IDENTIFIER.name());
+             throw new HandshakeException("Missing Property " + HandshakeProperty.PORT_IDENTIFIER.name());
+         }
+         
+         // send "OK" response
+         if ( !responseWritten ) {
+             ResponseCode.PROPERTIES_OK.writeResponse(dos);
+         }
+         
+         logger.debug("{} Finished handshake with {}", this, peer);
+         handshakeCompleted = true;
+     }
+     
+     @Override
+     public boolean isHandshakeSuccessful() {
+         return handshakeCompleted;
+     }
+     
+     @Override
+     public RootGroupPort getPort() {
+         return port;
+     }
+     
+     @Override
+     public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
+         if ( !handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has not been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} Negotiating Codec with {} using {}", new Object[] {this, peer, peer.getCommunicationsSession()});
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         
+         // Negotiate the FlowFileCodec to use.
+         try {
+             negotiatedFlowFileCodec = RemoteResourceFactory.receiveCodecNegotiation(dis, dos);
+             logger.debug("{} Negotiated Codec {} with {}", new Object[] {this, negotiatedFlowFileCodec, peer});
+             return negotiatedFlowFileCodec;
+         } catch (final HandshakeException e) {
+             throw new ProtocolException(e.toString());
+         }
+     }
+ 
+     @Override
+     public FlowFileCodec getPreNegotiatedCodec() {
+         return negotiatedFlowFileCodec;
+     }
+ 
+     
+     @Override
+     public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+         if ( !handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has not been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} Sending FlowFiles to {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         String remoteDn = commsSession.getUserDn();
+         if ( remoteDn == null ) {
+             remoteDn = "none";
+         }
+ 
+         FlowFile flowFile = session.get();
+         if ( flowFile == null ) {
+             // we have no data to send. Notify the peer.
+             logger.debug("{} No data to send to {}", this, peer);
+             ResponseCode.NO_MORE_DATA.writeResponse(dos);
+             return 0;
+         }
+         
+         // we have data to send.
+         logger.debug("{} Data is available to send to {}", this, peer);
+         ResponseCode.MORE_DATA.writeResponse(dos);
+         
+         final StopWatch stopWatch = new StopWatch(true);
+         long bytesSent = 0L;
+         final Set<FlowFile> flowFilesSent = new HashSet<>();
+         final CRC32 crc = new CRC32();
+ 
+         // send data until we reach some batch size
+         boolean continueTransaction = true;
+         final long startNanos = System.nanoTime();
+         String calculatedCRC = "";
+         while (continueTransaction) {
+             final OutputStream flowFileOutputStream = useGzip ? new CompressionOutputStream(dos) : dos;
+             logger.debug("{} Sending {} to {}", new Object[] {this, flowFile, peer});
+             
+             final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc);
+ 
+             final StopWatch transferWatch = new StopWatch(true);
+             flowFile = codec.encode(flowFile, session, checkedOutputStream);
+             final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS);
+             
+             // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
+             // Otherwise, do NOT close it because we don't want to close the underlying stream
+             // (CompressionOutputStream will not close the underlying stream when it's closed)
+             if ( useGzip ) {
+                 checkedOutputStream.close();
+             }
+ 
+             flowFilesSent.add(flowFile);
+             bytesSent += flowFile.getSize();
+ 
+             final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
+             session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false);
+             session.remove(flowFile);
+             
+             final long sendingNanos = System.nanoTime() - startNanos;
+             if ( sendingNanos < BATCH_NANOS ) { 
+                 flowFile = session.get();
+             } else {
+                 flowFile = null;
+             }
+             
+             continueTransaction = (flowFile != null);
+             if ( continueTransaction ) {
+                 logger.debug("{} Sending ContinueTransaction indicator to {}", this, peer);
+                 ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+             } else {
+                 logger.debug("{} Sending FinishTransaction indicator to {}", this, peer);
+                 ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+                 calculatedCRC = String.valueOf(checkedOutputStream.getChecksum().getValue());
+             }
+         }
+         
+         // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+         final Response transactionConfirmationResponse = Response.read(dis);
+         if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
+             // Confirm Checksum and echo back the confirmation.
+             logger.debug("{} Received {}  from {}", this, transactionConfirmationResponse, peer);
+             final String receivedCRC = transactionConfirmationResponse.getMessage();
+ 
+             if ( versionNegotiator.getVersion() > 3 ) {
+                 if ( !receivedCRC.equals(calculatedCRC) ) {
+                     ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+                     session.rollback();
+                     throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
+                 }
+             }
+ 
+             ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+         } else {
+             throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+         }
+ 
+         final String flowFileDescription = flowFilesSent.size() < 20 ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+         
+         final Response transactionResponse;
+         try {
+             transactionResponse = Response.read(dis);
+         } catch (final IOException e) {
+             logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
+                 " It is unknown whether or not the peer successfully received/processed the data." +
+                 " Therefore, {} will be rolled back, possibly resulting in data duplication of {}", 
+                 this, peer, session, flowFileDescription);
+             session.rollback();
+             throw e;
+         }
+         
+         logger.debug("{} received {} from {}", new Object[] {this, transactionResponse, peer});
+         if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+             peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
+         } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+             throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+         }
+         
+         session.commit();
+         
+         stopWatch.stop();
+         final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
+         final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+         final String dataSize = FormatUtils.formatDataSize(bytesSent);
+         logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+             this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ 
+         return flowFilesSent.size();
+     }
+     
+     
+     @Override
+     public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+         if ( !handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has not been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} receiving FlowFiles from {}", this, peer);
+         
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+         String remoteDn = commsSession.getUserDn();
+         if ( remoteDn == null ) {
+             remoteDn = "none";
+         }
+ 
+         final StopWatch stopWatch = new StopWatch(true);
+         final CRC32 crc = new CRC32();
+         
+         // Peer has data. Otherwise, we would not have been called, because they would not have sent
+         // a SEND_FLOWFILES request to use. Just decode the bytes into FlowFiles until peer says he's
+         // finished sending data.
+         final Set<FlowFile> flowFilesReceived = new HashSet<>();
+         long bytesReceived = 0L;
+         boolean continueTransaction = true;
+         String calculatedCRC = "";
+         while (continueTransaction) {
+             final long startNanos = System.nanoTime();
+             final InputStream flowFileInputStream = useGzip ? new CompressionInputStream(dis) : dis;
+             final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
+ 
+             FlowFile flowFile = codec.decode(checkedInputStream, session);
+             final long transferNanos = System.nanoTime() - startNanos;
+             final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+             final String sourceSystemFlowFileUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+             flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+             
+             final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceSystemFlowFileUuid;
+             session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceSystemFlowFileUuid, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transferMillis);
+             session.transfer(flowFile, Relationship.ANONYMOUS);
+             flowFilesReceived.add(flowFile);
+             bytesReceived += flowFile.getSize();
+             
+             final Response transactionResponse = Response.read(dis);
+             switch (transactionResponse.getCode()) {
+                 case CONTINUE_TRANSACTION:
+                     logger.debug("{} Received ContinueTransaction indicator from {}", this, peer);
+                     break;
+                 case FINISH_TRANSACTION:
+                     logger.debug("{} Received FinishTransaction indicator from {}", this, peer);
+                     continueTransaction = false;
+                     calculatedCRC = String.valueOf(checkedInputStream.getChecksum().getValue());
+                     break;
+                 default:
+                     throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse);
+             }
+         }
+         
+         // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+         // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+         // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+         // session and then when we send the response back to the peer, the peer may have timed out and may not
+         // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+         // Critical Section involved in this transaction so that rather than the Critical Section being the
+         // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+         logger.debug("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
+         ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+         
+         final Response confirmTransactionResponse = Response.read(dis);
+         logger.debug("{} Received {} from {}", this, confirmTransactionResponse, peer);
+ 
+         switch (confirmTransactionResponse.getCode()) {
+             case CONFIRM_TRANSACTION:
+                 break;
+             case BAD_CHECKSUM:
+                 session.rollback();
+                 throw new IOException(this + " Received a BadChecksum response from peer " + peer);
+             default:
+                 throw new ProtocolException(this + " Received unexpected Response Code from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
+         }
+         
+         // Commit the session so that we have persisted the data
+         session.commit();
+         
 -        if ( session.getAvailableRelationships().isEmpty() ) {
++        if ( context.getAvailableRelationships().isEmpty() ) {
+             // Confirm that we received the data and the peer can now discard it but that the peer should not
+             // send any more data for a bit
+             logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+             ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
+         } else {
+             // Confirm that we received the data and the peer can now discard it
+             logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
+             ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
+         }
+         
+         stopWatch.stop();
+         final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
+         final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
+         final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+         final String dataSize = FormatUtils.formatDataSize(bytesReceived);
+         logger.info("{} Successfully received {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
+             this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ 
+         return flowFilesReceived.size();
+     }
+     
+     @Override
+     public RequestType getRequestType(final Peer peer) throws IOException {
+         if ( !handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has not been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} Reading Request Type from {} using {}", new Object[] {this, peer, peer.getCommunicationsSession()});
+         final RequestType requestType = RequestType.readRequestType(new DataInputStream(peer.getCommunicationsSession().getInput().getInputStream()));
+         logger.debug("{} Got Request Type {} from {}", new Object[] {this, requestType, peer});
+ 
+         return requestType;
+     }
+ 
+     @Override
+     public VersionNegotiator getVersionNegotiator() {
+         return versionNegotiator;
+     }
+ 
+     @Override
+     public void shutdown(final Peer peer) {
+         logger.debug("{} Shutting down with {}", this, peer);
+         shutdown = true;
+     }
+ 
+     @Override
+     public boolean isShutdown() {
+         return shutdown;
+     }
+ 
+     @Override
+     public void sendPeerList(final Peer peer) throws IOException {
+         if ( !handshakeCompleted ) {
+             throw new IllegalStateException("Handshake has not been completed");
+         }
+         if ( shutdown ) {
+             throw new IllegalStateException("Protocol is shutdown");
+         }
+ 
+         logger.debug("{} Sending Peer List to {}", this, peer);
+         final CommunicationsSession commsSession = peer.getCommunicationsSession();
+         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ 
+         final NiFiProperties properties = NiFiProperties.getInstance();
+         
+         // we have only 1 peer: ourselves.
+         dos.writeInt(1);
+         dos.writeUTF(InetAddress.getLocalHost().getHostName());
+         dos.writeInt(properties.getRemoteInputPort());
+         dos.writeBoolean(properties.isSiteToSiteSecure());
+         dos.writeInt(0);    // doesn't matter how many FlowFiles we have, because we're the only host.
+         dos.flush();
+     }
+     
+     @Override
+     public String getResourceName() {
+         return RESOURCE_NAME;
+     }
+     
+     @Override
+     public void setNodeInformant(final NodeInformant nodeInformant) {
+     }
+ 
+     @Override
+     public long getRequestExpiration() {
+         return requestExpirationMillis;
+     }
+     
+     @Override
+     public String toString() {
+         return "SocketFlowFileServerProtocol[CommsID=" + commsIdentifier + "]";
+     }
+ }


[7/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
index 0000000,e516f20..e34e043
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@@ -1,0 -1,541 +1,541 @@@
+ /*
+  * 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.controller;
+ 
+ import static java.util.Objects.requireNonNull;
+ 
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReadWriteLock;
+ import java.util.concurrent.locks.ReentrantReadWriteLock;
+ 
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.connectable.Connectable;
+ import org.apache.nifi.connectable.ConnectableType;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.connectable.Funnel;
+ import org.apache.nifi.connectable.Position;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.groups.ProcessGroup;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.scheduling.SchedulingStrategy;
+ import org.apache.nifi.util.FormatUtils;
+ 
+ import org.apache.commons.lang3.builder.ToStringBuilder;
+ import org.apache.commons.lang3.builder.ToStringStyle;
+ 
+ public class StandardFunnel implements Funnel {
+ 
+     public static final long MINIMUM_PENALIZATION_MILLIS = 0L;
+     public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
+     public static final long MINIMUM_YIELD_MILLIS = 0L;
+     public static final long DEFAULT_YIELD_PERIOD = 1000L;
+     public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS;
+ 
+     private final String identifier;
+     private final Set<Connection> outgoingConnections;
+     private final List<Connection> incomingConnections;
+     private final List<Relationship> relationships;
+ 
+     private final AtomicReference<ProcessGroup> processGroupRef;
+     private final AtomicReference<Position> position;
+     private final AtomicReference<String> penalizationPeriod;
+     private final AtomicReference<String> yieldPeriod;
+     private final AtomicReference<String> schedulingPeriod;
+     private final AtomicReference<String> name;
+     private final AtomicLong schedulingNanos;
+     private final AtomicBoolean lossTolerant;
+     private final AtomicReference<ScheduledState> scheduledState;
+     private final AtomicLong yieldExpiration;
+ 
+     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+     private final Lock readLock = rwLock.readLock();
+     private final Lock writeLock = rwLock.writeLock();
+ 
+     public StandardFunnel(final String identifier, final ProcessGroup processGroup, final ProcessScheduler scheduler) {
+         this.identifier = identifier;
+         this.processGroupRef = new AtomicReference<>(processGroup);
+ 
+         outgoingConnections = new HashSet<>();
+         incomingConnections = new ArrayList<>();
+ 
+         final List<Relationship> relationships = new ArrayList<>();
+         relationships.add(Relationship.ANONYMOUS);
+         this.relationships = Collections.unmodifiableList(relationships);
+ 
+         lossTolerant = new AtomicBoolean(false);
+         position = new AtomicReference<>(new Position(0D, 0D));
+         scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
+         penalizationPeriod = new AtomicReference<>("30 sec");
+         yieldPeriod = new AtomicReference<>("1 sec");
+         yieldExpiration = new AtomicLong(0L);
+         schedulingPeriod = new AtomicReference<>("0 millis");
+         schedulingNanos = new AtomicLong(30000);
+         name = new AtomicReference<>("Funnel");
+     }
+ 
+     @Override
+     public String getIdentifier() {
+         return identifier;
+     }
+ 
+     @Override
+     public Collection<Relationship> getRelationships() {
+         return relationships;
+     }
+ 
+     @Override
+     public Relationship getRelationship(final String relationshipName) {
+         return (Relationship.ANONYMOUS.getName().equals(relationshipName)) ? Relationship.ANONYMOUS : null;
+     }
+ 
+     @Override
+     public void addConnection(final Connection connection) throws IllegalArgumentException {
+         writeLock.lock();
+         try {
+             if (!requireNonNull(connection).getSource().equals(this) && !connection.getDestination().equals(this)) {
+                 throw new IllegalArgumentException("Cannot add a connection to a Funnel for which the Funnel is neither the Source nor the Destination");
+             }
+             if (connection.getSource().equals(this) && connection.getDestination().equals(this)) {
+                 throw new IllegalArgumentException("Cannot add a connection from a Funnel back to itself");
+             }
+ 
+             if (connection.getDestination().equals(this)) {
+                 // don't add the connection twice. This may occur if we have a self-loop because we will be told
+                 // to add the connection once because we are the source and again because we are the destination.
+                 if (!incomingConnections.contains(connection)) {
+                     incomingConnections.add(connection);
+                 }
+             }
+ 
+             if (connection.getSource().equals(this)) {
+                 // don't add the connection twice. This may occur if we have a self-loop because we will be told
+                 // to add the connection once because we are the source and again because we are the destination.
+                 if (!outgoingConnections.contains(connection)) {
+                     for (final Relationship relationship : connection.getRelationships()) {
+                         if (!relationship.equals(Relationship.ANONYMOUS)) {
+                             throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Funnels");
+                         }
+                     }
+ 
+                     outgoingConnections.add(connection);
+                 }
+             }
+         } finally {
+             writeLock.unlock();
+         }
+     }
+ 
+     @Override
+     public boolean hasIncomingConnection() {
+         readLock.lock();
+         try {
+             return !incomingConnections.isEmpty();
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     @Override
+     public void updateConnection(final Connection connection) throws IllegalStateException {
+         if (requireNonNull(connection).getSource().equals(this)) {
+             writeLock.lock();
+             try {
+                 if (!outgoingConnections.remove(connection)) {
+                     throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
+                 }
+                 outgoingConnections.add(connection);
+             } finally {
+                 writeLock.unlock();
+             }
+         }
+ 
+         if (connection.getDestination().equals(this)) {
+             writeLock.lock();
+             try {
+                 if (!incomingConnections.remove(connection)) {
+                     throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
+                 }
+                 incomingConnections.add(connection);
+             } finally {
+                 writeLock.unlock();
+             }
+         }
+     }
+ 
+     @Override
+     public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException {
+         writeLock.lock();
+         try {
+             if (!requireNonNull(connection).getSource().equals(this)) {
+                 final boolean existed = incomingConnections.remove(connection);
+                 if (!existed) {
+                     throw new IllegalStateException("The given connection is not currently registered for this ProcessorNode");
+                 }
+                 return;
+             }
+ 
+             final boolean removed = outgoingConnections.remove(connection);
+             if (!removed) {
+                 throw new IllegalStateException(connection + " is not registered with " + this);
+             }
+         } finally {
+             writeLock.unlock();
+         }
+     }
+ 
+     @Override
+     public Set<Connection> getConnections() {
+         readLock.lock();
+         try {
+             return Collections.unmodifiableSet(outgoingConnections);
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     @Override
+     public Set<Connection> getConnections(final Relationship relationship) {
+         readLock.lock();
+         try {
+             if (relationship.equals(Relationship.ANONYMOUS)) {
+                 return Collections.unmodifiableSet(outgoingConnections);
+             }
+ 
+             throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Funnels");
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     @Override
+     public List<Connection> getIncomingConnections() {
+         readLock.lock();
+         try {
+             return new ArrayList<>(incomingConnections);
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     @Override
+     public Position getPosition() {
+         return position.get();
+     }
+ 
+     @Override
+     public void setPosition(Position position) {
+         this.position.set(position);
+     }
+ 
+     @Override
+     public String getName() {
+         return name.get();
+     }
+ 
+     /**
+      * Throws {@link UnsupportedOperationException}
+      *
+      * @param name
+      */
+     @Override
+     public void setName(final String name) {
+         throw new UnsupportedOperationException();
+     }
+ 
+     @Override
+     public String getComments() {
+         return "";
+     }
+ 
+     @Override
+     public void setComments(final String comments) {
+         throw new UnsupportedOperationException();
+     }
+ 
+     @Override
+     public ProcessGroup getProcessGroup() {
+         return processGroupRef.get();
+     }
+ 
+     @Override
+     public void setProcessGroup(final ProcessGroup group) {
+         processGroupRef.set(group);
+     }
+ 
+     @Override
+     public boolean isAutoTerminated(Relationship relationship) {
+         return false;
+     }
+ 
+     @Override
+     public boolean isRunning() {
+         return isRunning(this);
+     }
+ 
+     private boolean isRunning(final Connectable source) {
+         return getScheduledState() == ScheduledState.RUNNING;
+     }
+ 
+     @Override
+     public boolean isTriggerWhenEmpty() {
+         return false;
+     }
+ 
+     @Override
+     public ScheduledState getScheduledState() {
+         return scheduledState.get();
+     }
+ 
+     @Override
+     public boolean isLossTolerant() {
+         return lossTolerant.get();
+     }
+ 
+     @Override
+     public void setLossTolerant(final boolean lossTolerant) {
+         this.lossTolerant.set(lossTolerant);
+     }
+ 
+     @Override
+     public String toString() {
+         return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("id", getIdentifier()).toString();
+     }
+ 
+     @Override
+     public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+         final ProcessSession session = sessionFactory.createSession();
+ 
+         try {
+             onTrigger(context, session);
+             session.commit();
+         } catch (final ProcessException e) {
+             session.rollback();
+             throw e;
+         } catch (final Throwable t) {
+             session.rollback();
+             throw new RuntimeException(t);
+         }
+     }
+ 
+     private void onTrigger(final ProcessContext context, final ProcessSession session) {
+         readLock.lock();
+         try {
 -            Set<Relationship> available = session.getAvailableRelationships();
++            Set<Relationship> available = context.getAvailableRelationships();
+             int transferred = 0;
+             while (!available.isEmpty()) {
+                 final List<FlowFile> flowFiles = session.get(10);
+                 if (flowFiles.isEmpty()) {
+                     break;
+                 }
+ 
+                 transferred += flowFiles.size();
+                 session.transfer(flowFiles, Relationship.ANONYMOUS);
+                 session.commit();
 -                available = session.getAvailableRelationships();
++                available = context.getAvailableRelationships();
+             }
+ 
+             if (transferred == 0) {
+                 context.yield();
+             }
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     /**
+      * Has no effect
+      */
+     @Override
+     public void setMaxConcurrentTasks(int taskCount) {
+     }
+ 
+     @Override
+     public int getMaxConcurrentTasks() {
+         return 1;
+     }
+ 
+     @Override
+     public void setScheduledState(final ScheduledState scheduledState) {
+         this.scheduledState.set(scheduledState);
+     }
+ 
+     @Override
+     public ConnectableType getConnectableType() {
+         return ConnectableType.FUNNEL;
+     }
+ 
+     @Override
+     @SuppressWarnings("unchecked")
+     public Collection<ValidationResult> getValidationErrors() {
+         return Collections.EMPTY_LIST;
+     }
+ 
+     /**
+      * Updates the amount of time that this processor should avoid being
+      * scheduled when the processor calls
+      * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
+      *
+      * @param yieldPeriod
+      */
+     @Override
+     public void setYieldPeriod(final String yieldPeriod) {
+         final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS);
+         if (yieldMillis < 0) {
+             throw new IllegalArgumentException("Yield duration must be positive");
+         }
+         this.yieldPeriod.set(yieldPeriod);
+     }
+ 
+     /**
+      * @param schedulingPeriod
+      */
+     @Override
+     public void setScheduldingPeriod(final String schedulingPeriod) {
+         final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS);
+         if (schedulingNanos < 0) {
+             throw new IllegalArgumentException("Scheduling Period must be positive");
+         }
+ 
+         this.schedulingPeriod.set(schedulingPeriod);
+         this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos));
+     }
+ 
+     @Override
+     public long getPenalizationPeriod(final TimeUnit timeUnit) {
+         return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
+     }
+ 
+     @Override
+     public String getPenalizationPeriod() {
+         return penalizationPeriod.get();
+     }
+ 
+     /**
+      * Causes the processor not to be scheduled for some period of time. This
+      * duration can be obtained and set via the
+      * {@link #getYieldPeriod(TimeUnit)} and
+      * {@link #setYieldPeriod(long, TimeUnit)} methods.
+      */
+     @Override
+     public void yield() {
+         final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
+         yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis));
+     }
+ 
+     @Override
+     public long getYieldExpiration() {
+         return yieldExpiration.get();
+     }
+ 
+     @Override
+     public String getSchedulingPeriod() {
+         return schedulingPeriod.get();
+     }
+ 
+     @Override
+     public void setPenalizationPeriod(final String penalizationPeriod) {
+         this.penalizationPeriod.set(penalizationPeriod);
+     }
+ 
+     @Override
+     public String getYieldPeriod() {
+         return yieldPeriod.get();
+     }
+ 
+     @Override
+     public long getYieldPeriod(final TimeUnit timeUnit) {
+         return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
+     }
+ 
+     @Override
+     public long getSchedulingPeriod(final TimeUnit timeUnit) {
+         return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS);
+     }
+ 
+     @Override
+     public boolean isSideEffectFree() {
+         return true;
+     }
+ 
+     @Override
+     public void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException {
+         if (ignoreConnections) {
+             return;
+         }
+ 
+         readLock.lock();
+         try {
+             for (final Connection connection : outgoingConnections) {
+                 connection.verifyCanDelete();
+             }
+ 
+             for (final Connection connection : incomingConnections) {
+                 if (connection.getSource().equals(this)) {
+                     connection.verifyCanDelete();
+                 } else {
+                     throw new IllegalStateException(this + " is the destination of another component");
+                 }
+             }
+         } finally {
+             readLock.unlock();
+         }
+     }
+ 
+     @Override
+     public void verifyCanDelete() {
+         verifyCanDelete(false);
+     }
+ 
+     @Override
+     public void verifyCanStart() {
+     }
+ 
+     @Override
+     public void verifyCanStop() {
+     }
+ 
+     @Override
+     public void verifyCanUpdate() {
+     }
+ 
+     @Override
+     public void verifyCanEnable() {
+     }
+ 
+     @Override
+     public void verifyCanDisable() {
+     }
+ 
+     @Override
+     public SchedulingStrategy getSchedulingStrategy() {
+         return SchedulingStrategy.TIMER_DRIVEN;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index 0000000,eae2550..d5dba82
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@@ -1,0 -1,247 +1,242 @@@
+ /*
+  * 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.controller.repository;
+ 
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Path;
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.regex.Pattern;
+ 
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ 
+ public class BatchingSessionFactory implements ProcessSessionFactory {
+ 
+     private final HighThroughputSession highThroughputSession;
+ 
+     public BatchingSessionFactory(final StandardProcessSession standardProcessSession) {
+         highThroughputSession = new HighThroughputSession(standardProcessSession);
+     }
+ 
+     @Override
+     public ProcessSession createSession() {
+         return highThroughputSession;
+     }
+ 
+     private class HighThroughputSession implements ProcessSession {
+ 
+         private final StandardProcessSession session;
+ 
+         public HighThroughputSession(final StandardProcessSession session) {
+             this.session = session;
+         }
+ 
+         @Override
+         public void commit() {
+             session.checkpoint();
+         }
+ 
+         @Override
+         public void rollback() {
+             session.rollback();
+         }
+ 
+         @Override
+         public void rollback(boolean penalize) {
+             session.rollback(penalize);
+         }
+ 
+         @Override
+         public void adjustCounter(String name, long delta, boolean immediate) {
+             session.adjustCounter(name, delta, immediate);
+         }
+ 
+         @Override
+         public FlowFile get() {
+             return session.get();
+         }
+ 
+         @Override
+         public List<FlowFile> get(int maxResults) {
+             return session.get(maxResults);
+         }
+ 
+         @Override
+         public List<FlowFile> get(FlowFileFilter filter) {
+             return session.get(filter);
+         }
+ 
+         @Override
+         public QueueSize getQueueSize() {
+             return session.getQueueSize();
+         }
+ 
+         @Override
 -        public Set<Relationship> getAvailableRelationships() {
 -            return session.getAvailableRelationships();
 -        }
 -
 -        @Override
+         public FlowFile create() {
+             return session.create();
+         }
+ 
+         @Override
+         public FlowFile create(FlowFile parent) {
+             return session.create(parent);
+         }
+ 
+         @Override
+         public FlowFile create(Collection<FlowFile> parents) {
+             return session.create(parents);
+         }
+ 
+         @Override
+         public FlowFile clone(FlowFile example) {
+             return session.clone(example);
+         }
+ 
+         @Override
+         public FlowFile clone(FlowFile example, long offset, long size) {
+             return session.clone(example, offset, size);
+         }
+ 
+         @Override
+         public FlowFile penalize(FlowFile flowFile) {
+             return session.penalize(flowFile);
+         }
+ 
+         @Override
+         public FlowFile putAttribute(FlowFile flowFile, String key, String value) {
+             return session.putAttribute(flowFile, key, value);
+         }
+ 
+         @Override
+         public FlowFile putAllAttributes(FlowFile flowFile, Map<String, String> attributes) {
+             return session.putAllAttributes(flowFile, attributes);
+         }
+ 
+         @Override
+         public FlowFile removeAttribute(FlowFile flowFile, String key) {
+             return session.removeAttribute(flowFile, key);
+         }
+ 
+         @Override
+         public FlowFile removeAllAttributes(FlowFile flowFile, Set<String> keys) {
+             return session.removeAllAttributes(flowFile, keys);
+         }
+ 
+         @Override
+         public FlowFile removeAllAttributes(FlowFile flowFile, Pattern keyPattern) {
+             return session.removeAllAttributes(flowFile, keyPattern);
+         }
+ 
+         @Override
+         public void transfer(FlowFile flowFile, Relationship relationship) {
+             session.transfer(flowFile, relationship);
+         }
+ 
+         @Override
+         public void transfer(FlowFile flowFile) {
+             session.transfer(flowFile);
+         }
+ 
+         @Override
+         public void transfer(Collection<FlowFile> flowFiles) {
+             session.transfer(flowFiles);
+         }
+ 
+         @Override
+         public void transfer(Collection<FlowFile> flowFiles, Relationship relationship) {
+             session.transfer(flowFiles, relationship);
+         }
+ 
+         @Override
+         public void remove(FlowFile flowFile) {
+             session.remove(flowFile);
+         }
+ 
+         @Override
+         public void remove(Collection<FlowFile> flowFiles) {
+             session.remove(flowFiles);
+         }
+ 
+         @Override
+         public void read(FlowFile source, InputStreamCallback reader) {
+             session.read(source, reader);
+         }
+ 
+         @Override
+         public FlowFile merge(Collection<FlowFile> sources, FlowFile destination) {
+             return session.merge(sources, destination);
+         }
+ 
+         @Override
+         public FlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator) {
+             return session.merge(sources, destination, header, footer, demarcator);
+         }
+ 
+         @Override
+         public FlowFile write(FlowFile source, OutputStreamCallback writer) {
+             return session.write(source, writer);
+         }
+ 
+         @Override
+         public FlowFile write(FlowFile source, StreamCallback writer) {
+             return session.write(source, writer);
+         }
+ 
+         @Override
+         public FlowFile append(FlowFile source, OutputStreamCallback writer) {
+             return session.append(source, writer);
+         }
+ 
+         @Override
+         public FlowFile importFrom(Path source, boolean keepSourceFile, FlowFile destination) {
+             return session.importFrom(source, keepSourceFile, destination);
+         }
+ 
+         @Override
+         public FlowFile importFrom(InputStream source, FlowFile destination) {
+             return session.importFrom(source, destination);
+         }
+ 
+         @Override
+         public void exportTo(FlowFile flowFile, Path destination, boolean append) {
+             session.exportTo(flowFile, destination, append);
+         }
+ 
+         @Override
+         public void exportTo(FlowFile flowFile, OutputStream destination) {
+             session.exportTo(flowFile, destination);
+         }
+ 
+         @Override
+         public ProvenanceReporter getProvenanceReporter() {
+             return session.getProvenanceReporter();
+         }
+ 
+     }
+ 
+ }


[8/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


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

Branch: refs/heads/develop
Commit: 73384b23d1de7c814cd6b8089de7c893db057244
Parents: 3a7b8de db23609
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Jan 16 11:29:35 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Jan 16 11:29:35 2015 -0500

----------------------------------------------------------------------
 assembly/pom.xml                                |  458 ---
 assembly/src/main/assembly/dependencies.xml     |  140 -
 commons/data-provenance-utils/.gitignore        |    2 -
 commons/data-provenance-utils/pom.xml           |   40 -
 .../nifi/provenance/AsyncLineageSubmission.java |   87 -
 .../nifi/provenance/AsyncQuerySubmission.java   |   81 -
 .../nifi/provenance/NamedSearchableField.java   |   95 -
 .../nifi/provenance/SearchableFieldParser.java  |   53 -
 .../nifi/provenance/SearchableFields.java       |   84 -
 .../nifi/provenance/StandardLineageResult.java  |  324 --
 .../StandardProvenanceEventRecord.java          |  752 ----
 .../nifi/provenance/StandardQueryResult.java    |  168 -
 .../nifi/provenance/lineage/EdgeNode.java       |   74 -
 .../nifi/provenance/lineage/EventNode.java      |  109 -
 .../provenance/lineage/FlowFileLineage.java     |   76 -
 .../nifi/provenance/lineage/FlowFileNode.java   |   83 -
 commons/flowfile-packager/pom.xml               |   41 -
 .../org/apache/nifi/util/FlowFilePackager.java  |   28 -
 .../apache/nifi/util/FlowFilePackagerV1.java    |  104 -
 .../apache/nifi/util/FlowFilePackagerV2.java    |  146 -
 .../apache/nifi/util/FlowFilePackagerV3.java    |   93 -
 .../apache/nifi/util/FlowFileUnpackager.java    |   30 -
 .../apache/nifi/util/FlowFileUnpackagerV1.java  |  155 -
 .../apache/nifi/util/FlowFileUnpackagerV2.java  |  143 -
 .../apache/nifi/util/FlowFileUnpackagerV3.java  |  161 -
 .../java/org/apache/nifi/util/Unpackage.java    |  119 -
 .../nifi/util/TestPackageUnpackageV3.java       |   56 -
 commons/nifi-expression-language/pom.xml        |   56 -
 .../language/antlr/AttributeExpressionLexer.g   |  210 -
 .../language/antlr/AttributeExpressionParser.g  |  139 -
 .../output/AttributeExpressionLexer.tokens      |   88 -
 .../expression/language/EmptyPreparedQuery.java |   62 -
 .../language/InvalidPreparedQuery.java          |   71 -
 .../expression/language/PreparedQuery.java      |   39 -
 .../attribute/expression/language/Query.java    | 1186 ------
 .../language/StandardAttributeExpression.java   |   65 -
 .../StandardExpressionLanguageCompiler.java     |   58 -
 .../language/StandardPreparedQuery.java         |   83 -
 .../language/evaluation/BooleanEvaluator.java   |   32 -
 .../language/evaluation/BooleanQueryResult.java |   43 -
 .../language/evaluation/DateEvaluator.java      |   34 -
 .../language/evaluation/DateQueryResult.java    |   45 -
 .../language/evaluation/Evaluator.java          |   32 -
 .../language/evaluation/NumberEvaluator.java    |   33 -
 .../language/evaluation/NumberQueryResult.java  |   43 -
 .../language/evaluation/QueryResult.java        |   26 -
 .../language/evaluation/StringEvaluator.java    |   32 -
 .../language/evaluation/StringQueryResult.java  |   43 -
 .../evaluation/cast/BooleanCastEvaluator.java   |   50 -
 .../evaluation/cast/DateCastEvaluator.java      |  117 -
 .../evaluation/cast/NumberCastEvaluator.java    |   72 -
 .../evaluation/cast/StringCastEvaluator.java    |   49 -
 .../evaluation/functions/AndEvaluator.java      |   60 -
 .../evaluation/functions/AppendEvaluator.java   |   50 -
 .../functions/AttributeEvaluator.java           |   45 -
 .../evaluation/functions/ContainsEvaluator.java |   53 -
 .../functions/DateToNumberEvaluator.java        |   50 -
 .../evaluation/functions/DivideEvaluator.java   |   57 -
 .../evaluation/functions/EndsWithEvaluator.java |   53 -
 .../evaluation/functions/EqualsEvaluator.java   |   89 -
 .../functions/EqualsIgnoreCaseEvaluator.java    |   60 -
 .../evaluation/functions/FindEvaluator.java     |   72 -
 .../evaluation/functions/FormatEvaluator.java   |   61 -
 .../functions/GreaterThanEvaluator.java         |   60 -
 .../functions/GreaterThanOrEqualEvaluator.java  |   60 -
 .../evaluation/functions/HostnameEvaluator.java |   58 -
 .../evaluation/functions/IPEvaluator.java       |   46 -
 .../evaluation/functions/IndexOfEvaluator.java  |   53 -
 .../evaluation/functions/IsEmptyEvaluator.java  |   43 -
 .../evaluation/functions/IsNullEvaluator.java   |   45 -
 .../functions/LastIndexOfEvaluator.java         |   53 -
 .../evaluation/functions/LengthEvaluator.java   |   46 -
 .../evaluation/functions/LessThanEvaluator.java |   60 -
 .../functions/LessThanOrEqualEvaluator.java     |   60 -
 .../evaluation/functions/MatchesEvaluator.java  |   71 -
 .../evaluation/functions/MinusEvaluator.java    |   57 -
 .../evaluation/functions/ModEvaluator.java      |   57 -
 .../evaluation/functions/MultiplyEvaluator.java |   57 -
 .../evaluation/functions/NotEvaluator.java      |   49 -
 .../evaluation/functions/NotNullEvaluator.java  |   45 -
 .../evaluation/functions/NowEvaluator.java      |   39 -
 .../functions/NumberToDateEvaluator.java        |   52 -
 .../functions/OneUpSequenceEvaluator.java       |   41 -
 .../evaluation/functions/OrEvaluator.java       |   60 -
 .../evaluation/functions/PlusEvaluator.java     |   57 -
 .../evaluation/functions/PrependEvaluator.java  |   50 -
 .../functions/ReplaceAllEvaluator.java          |   55 -
 .../functions/ReplaceEmptyEvaluator.java        |   50 -
 .../evaluation/functions/ReplaceEvaluator.java  |   55 -
 .../functions/ReplaceNullEvaluator.java         |   47 -
 .../functions/StartsWithEvaluator.java          |   53 -
 .../functions/StringToDateEvaluator.java        |   65 -
 .../functions/SubstringAfterEvaluator.java      |   59 -
 .../functions/SubstringAfterLastEvaluator.java  |   55 -
 .../functions/SubstringBeforeEvaluator.java     |   58 -
 .../functions/SubstringBeforeLastEvaluator.java |   55 -
 .../functions/SubstringEvaluator.java           |   65 -
 .../evaluation/functions/ToLowerEvaluator.java  |   45 -
 .../evaluation/functions/ToNumberEvaluator.java |   46 -
 .../evaluation/functions/ToRadixEvaluator.java  |   77 -
 .../evaluation/functions/ToStringEvaluator.java |   45 -
 .../evaluation/functions/ToUpperEvaluator.java  |   45 -
 .../evaluation/functions/TrimEvaluator.java     |   45 -
 .../functions/UrlDecodeEvaluator.java           |   55 -
 .../functions/UrlEncodeEvaluator.java           |   55 -
 .../evaluation/functions/UuidEvaluator.java     |   39 -
 .../literals/BooleanLiteralEvaluator.java       |   44 -
 .../literals/NumberLiteralEvaluator.java        |   44 -
 .../literals/StringLiteralEvaluator.java        |   77 -
 .../evaluation/reduce/CountEvaluator.java       |   56 -
 .../evaluation/reduce/JoinEvaluator.java        |   59 -
 .../evaluation/reduce/ReduceEvaluator.java      |   23 -
 .../selection/AllAttributesEvaluator.java       |   68 -
 .../selection/AnyAttributeEvaluator.java        |   68 -
 .../AnyMatchingAttributeEvaluator.java          |   21 -
 .../selection/DelineatedAttributeEvaluator.java |   83 -
 .../evaluation/selection/MappingEvaluator.java  |   61 -
 .../selection/MultiAttributeEvaluator.java      |   24 -
 .../selection/MultiMatchAttributeEvaluator.java |   82 -
 .../selection/MultiNamedAttributeEvaluator.java |   64 -
 .../AttributeExpressionLanguageException.java   |   34 -
 ...ibuteExpressionLanguageParsingException.java |   34 -
 .../exception/IllegalAttributeException.java    |   29 -
 .../expression/language/TestQuery.java          | 1143 ------
 .../language/TestStandardPreparedQuery.java     |   92 -
 commons/nifi-logging-utils/pom.xml              |   36 -
 .../java/org/apache/nifi/logging/NiFiLog.java   |  367 --
 commons/nifi-properties/.gitignore              |    3 -
 commons/nifi-properties/pom.xml                 |   29 -
 .../org/apache/nifi/util/NiFiProperties.java    |  876 -----
 .../java/org/apache/nifi/util/StringUtils.java  |   66 -
 commons/nifi-security-utils/pom.xml             |   40 -
 .../nifi/security/util/CertificateUtils.java    |  158 -
 .../nifi/security/util/EncryptionMethod.java    |   84 -
 .../apache/nifi/security/util/KeystoreType.java |   26 -
 .../nifi/security/util/SecurityStoreTypes.java  |  144 -
 .../nifi/security/util/SslContextFactory.java   |  180 -
 commons/nifi-socket-utils/pom.xml               |   60 -
 .../nifi/io/nio/AbstractChannelReader.java      |  166 -
 .../java/org/apache/nifi/io/nio/BufferPool.java |  114 -
 .../apache/nifi/io/nio/ChannelDispatcher.java   |  160 -
 .../org/apache/nifi/io/nio/ChannelListener.java |  228 --
 .../nifi/io/nio/DatagramChannelReader.java      |   59 -
 .../apache/nifi/io/nio/SocketChannelReader.java |   55 -
 .../io/nio/consumer/AbstractStreamConsumer.java |  132 -
 .../nifi/io/nio/consumer/StreamConsumer.java    |   80 -
 .../io/nio/consumer/StreamConsumerFactory.java  |   27 -
 .../nifi/io/socket/SSLContextFactory.java       |  102 -
 .../io/socket/ServerSocketConfiguration.java    |   83 -
 .../nifi/io/socket/SocketConfiguration.java     |  116 -
 .../apache/nifi/io/socket/SocketListener.java   |  211 -
 .../org/apache/nifi/io/socket/SocketUtils.java  |  169 -
 .../socket/multicast/DiscoverableService.java   |   43 -
 .../multicast/DiscoverableServiceImpl.java      |   78 -
 .../multicast/MulticastConfiguration.java       |   99 -
 .../io/socket/multicast/MulticastListener.java  |  193 -
 .../multicast/MulticastServiceDiscovery.java    |   34 -
 .../multicast/MulticastServicesBroadcaster.java |   33 -
 .../socket/multicast/MulticastTimeToLive.java   |   50 -
 .../io/socket/multicast/MulticastUtils.java     |  109 -
 .../io/socket/multicast/ServiceDiscovery.java   |   31 -
 .../socket/multicast/ServicesBroadcaster.java   |   56 -
 .../apache/nifi/io/nio/example/ServerMain.java  |  141 -
 .../apache/nifi/io/nio/example/TCPClient.java   |   86 -
 .../apache/nifi/io/nio/example/UDPClient.java   |   51 -
 .../io/nio/example/UselessStreamConsumer.java   |   43 -
 .../src/test/resources/log4j.xml                |   36 -
 commons/nifi-utils/.gitignore                   |    8 -
 commons/nifi-utils/pom.xml                      |   33 -
 .../flowfile/attributes/CoreAttributes.java     |   72 -
 .../attributes/FlowFileAttributeKey.java        |   21 -
 .../nifi/remote/StandardVersionNegotiator.java  |   81 -
 .../apache/nifi/remote/VersionNegotiator.java   |   65 -
 .../TransmissionDisabledException.java          |   25 -
 .../nifi/remote/io/CompressionInputStream.java  |  184 -
 .../nifi/remote/io/CompressionOutputStream.java |  147 -
 .../remote/io/InterruptableInputStream.java     |  117 -
 .../remote/io/InterruptableOutputStream.java    |   81 -
 .../remote/io/socket/BufferStateManager.java    |  111 -
 .../io/socket/SocketChannelInputStream.java     |  157 -
 .../io/socket/SocketChannelOutputStream.java    |  113 -
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  602 ---
 .../socket/ssl/SSLSocketChannelInputStream.java |   62 -
 .../ssl/SSLSocketChannelOutputStream.java       |   53 -
 .../nifi/stream/io/BufferedInputStream.java     |   37 -
 .../nifi/stream/io/BufferedOutputStream.java    |  140 -
 .../nifi/stream/io/ByteArrayInputStream.java    |  250 --
 .../nifi/stream/io/ByteArrayOutputStream.java   |  250 --
 .../nifi/stream/io/ByteCountingInputStream.java |  104 -
 .../stream/io/ByteCountingOutputStream.java     |   63 -
 .../apache/nifi/stream/io/DataOutputStream.java |  417 --
 .../apache/nifi/stream/io/GZIPOutputStream.java |   41 -
 .../stream/io/LeakyBucketStreamThrottler.java   |  331 --
 .../nifi/stream/io/NonCloseableInputStream.java |   56 -
 .../stream/io/NonCloseableOutputStream.java     |   51 -
 .../apache/nifi/stream/io/NullOutputStream.java |   46 -
 .../apache/nifi/stream/io/StreamThrottler.java  |   33 -
 .../org/apache/nifi/stream/io/StreamUtils.java  |  257 --
 .../apache/nifi/stream/io/ZipOutputStream.java  |   38 -
 .../exception/BytePatternNotFoundException.java |   28 -
 .../io/util/NonThreadSafeCircularBuffer.java    |   69 -
 .../org/apache/nifi/util/BooleanHolder.java     |   25 -
 .../java/org/apache/nifi/util/FormatUtils.java  |  205 -
 .../org/apache/nifi/util/IntegerHolder.java     |   54 -
 .../java/org/apache/nifi/util/LongHolder.java   |   60 -
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  135 -
 .../java/org/apache/nifi/util/ObjectHolder.java |   39 -
 .../java/org/apache/nifi/util/RingBuffer.java   |  292 --
 .../java/org/apache/nifi/util/StopWatch.java    |  127 -
 .../main/java/org/apache/nifi/util/Tuple.java   |   83 -
 .../concurrency/DebugDisabledTimedLock.java     |   67 -
 .../util/concurrency/DebugEnabledTimedLock.java |  136 -
 .../util/concurrency/DebuggableTimedLock.java   |   30 -
 .../apache/nifi/util/concurrency/TimedLock.java |   59 -
 .../org/apache/nifi/util/file/FileUtils.java    |  623 ---
 .../file/monitor/CompoundUpdateMonitor.java     |  115 -
 .../util/file/monitor/LastModifiedMonitor.java  |   30 -
 .../nifi/util/file/monitor/MD5SumMonitor.java   |   51 -
 .../file/monitor/SynchronousFileWatcher.java    |  123 -
 .../nifi/util/file/monitor/UpdateMonitor.java   |   25 -
 .../org/apache/nifi/util/search/Search.java     |   57 -
 .../org/apache/nifi/util/search/SearchTerm.java |  141 -
 .../util/search/ahocorasick/AhoCorasick.java    |  155 -
 .../nifi/util/search/ahocorasick/Node.java      |   72 -
 .../util/search/ahocorasick/SearchState.java    |   63 -
 .../nifi/util/timebuffer/EntityAccess.java      |   26 -
 .../nifi/util/timebuffer/LongEntityAccess.java  |   43 -
 .../nifi/util/timebuffer/TimedBuffer.java       |  114 -
 .../nifi/util/timebuffer/TimestampedLong.java   |   35 -
 .../io/TestCompressionInputOutputStreams.java   |  153 -
 .../stream/io/TestLeakyBucketThrottler.java     |  147 -
 .../nifi/util/TestNaiveSearchRingBuffer.java    |   72 -
 .../file/monitor/TestCompoundUpdateMonitor.java |   71 -
 .../monitor/TestSynchronousFileWatcher.java     |   61 -
 .../nifi/util/timebuffer/TestRingBuffer.java    |  182 -
 .../nifi/util/timebuffer/TestTimedBuffer.java   |  106 -
 .../src/test/resources/logback-test.xml         |   32 -
 commons/nifi-web-utils/pom.xml                  |   56 -
 .../org/apache/nifi/web/util/ClientUtils.java   |  132 -
 .../nifi/web/util/ObjectMapperResolver.java     |   48 -
 .../java/org/apache/nifi/web/util/WebUtils.java |  198 -
 commons/pom.xml                                 |   44 -
 commons/processor-utilities/pom.xml             |   45 -
 .../nifi/processor/util/FlowFileFilters.java    |   65 -
 .../nifi/processor/util/SSLProperties.java      |  226 --
 .../nifi/processor/util/StandardValidators.java |  544 ---
 .../apache/nifi/processor/TestFormatUtils.java  |   40 -
 .../processor/util/TestStandardValidators.java  |   54 -
 commons/wali/.gitignore                         |    2 -
 commons/wali/pom.xml                            |   41 -
 .../org/wali/MinimalLockingWriteAheadLog.java   | 1008 -----
 commons/wali/src/main/java/org/wali/SerDe.java  |  128 -
 .../src/main/java/org/wali/SyncListener.java    |   62 -
 .../wali/src/main/java/org/wali/UpdateType.java |   49 -
 .../java/org/wali/WriteAheadRepository.java     |  122 -
 .../src/test/java/org/wali/DummyRecord.java     |   61 -
 .../test/java/org/wali/DummyRecordSerde.java    |  107 -
 .../wali/TestMinimalLockingWriteAheadLog.java   |  298 --
 maven-plugins/nar-maven-plugin/pom.xml          |   75 +
 .../src/main/java/nifi/NarMojo.java             |  613 +++
 .../resources/META-INF/plexus/components.xml    |   52 +
 maven-plugins/pom.xml                           |  346 ++
 .../execute-script-processors/pom.xml           |   81 -
 .../nifi/processors/script/ExecuteScript.java   |  566 ---
 .../apache/nifi/scripting/ConverterScript.java  |  131 -
 .../nifi/scripting/JRubyScriptFactory.java      |   46 -
 .../nifi/scripting/JavaScriptScriptFactory.java |   56 -
 .../nifi/scripting/JythonScriptFactory.java     |   45 -
 .../nifi/scripting/OutputStreamHandler.java     |   24 -
 .../org/apache/nifi/scripting/ReaderScript.java |   79 -
 .../java/org/apache/nifi/scripting/Script.java  |  303 --
 .../nifi/scripting/ScriptEngineFactory.java     |  117 -
 .../apache/nifi/scripting/ScriptFactory.java    |  269 --
 .../org/apache/nifi/scripting/WriterScript.java |   67 -
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  264 --
 .../processors/script/TestExecuteScript.java    |  939 -----
 .../src/test/resources/alwaysFail.js            |   24 -
 .../src/test/resources/alwaysFail.py            |   19 -
 .../src/test/resources/alwaysFail.rb            |   21 -
 .../src/test/resources/ffTest.js                |   28 -
 .../src/test/resources/ffTest.py                |   22 -
 .../src/test/resources/ffTest.rb                |   30 -
 .../src/test/resources/lib/Sub.py               |   18 -
 .../src/test/resources/lib/sub.js               |   22 -
 .../src/test/resources/lib/sub.rb               |   17 -
 .../src/test/resources/loadLocal.js             |   30 -
 .../src/test/resources/loadLocal.py             |   26 -
 .../src/test/resources/loadLocal.rb             |   29 -
 .../src/test/resources/log4j.xml                |   54 -
 .../src/test/resources/optionalValidators.js    |   28 -
 .../src/test/resources/optionalValidators.py    |   22 -
 .../src/test/resources/optionalValidators.rb    |   39 -
 .../src/test/resources/paramTest.js             |   28 -
 .../src/test/resources/paramTest.py             |   26 -
 .../src/test/resources/paramTest.rb             |   31 -
 .../src/test/resources/parseXml.js              |   36 -
 .../src/test/resources/readTest.js              |   30 -
 .../src/test/resources/readTest.py              |   32 -
 .../src/test/resources/readTest.rb              |   30 -
 .../src/test/resources/readWithParams.js        |   32 -
 .../src/test/resources/readWithParams.py        |   32 -
 .../src/test/resources/readWithParams.rb        |   33 -
 .../src/test/resources/routeTest.js             |   41 -
 .../src/test/resources/routeTest.py             |   37 -
 .../src/test/resources/routeTest.rb             |   39 -
 .../src/test/resources/simpleConverter.js       |   45 -
 .../src/test/resources/simpleConverter.py       |   60 -
 .../src/test/resources/simpleConverter.rb       |   42 -
 .../src/test/resources/writeTest.js             |   26 -
 .../src/test/resources/writeTest.py             |   22 -
 .../src/test/resources/writeTest.rb             |   32 -
 nar-bundles/execute-script-bundle/nar/pom.xml   |   36 -
 nar-bundles/execute-script-bundle/pom.xml       |   81 -
 .../framework/administration/.gitignore         |    1 -
 .../framework/administration/pom.xml            |  116 -
 .../nifi/admin/AuditDataSourceFactoryBean.java  |  222 --
 .../org/apache/nifi/admin/RepositoryUtils.java  |   91 -
 .../nifi/admin/UserDataSourceFactoryBean.java   |  247 --
 .../org/apache/nifi/admin/dao/ActionDAO.java    |   74 -
 .../org/apache/nifi/admin/dao/AuthorityDAO.java |   58 -
 .../org/apache/nifi/admin/dao/DAOFactory.java   |   29 -
 .../nifi/admin/dao/DataAccessException.java     |   39 -
 .../java/org/apache/nifi/admin/dao/UserDAO.java |  127 -
 .../nifi/admin/dao/impl/DAOFactoryImpl.java     |   51 -
 .../nifi/admin/dao/impl/StandardActionDAO.java  | 1056 -----
 .../admin/dao/impl/StandardAuthorityDAO.java    |  172 -
 .../nifi/admin/dao/impl/StandardUserDAO.java    |  634 ---
 .../admin/service/AccountDisabledException.java |   40 -
 .../admin/service/AccountNotFoundException.java |   40 -
 .../admin/service/AccountPendingException.java  |   41 -
 .../admin/service/AdministrationException.java  |   39 -
 .../apache/nifi/admin/service/AuditService.java |   76 -
 .../apache/nifi/admin/service/UserService.java  |  161 -
 .../service/action/AbstractUserAction.java      |   97 -
 .../admin/service/action/AddActionsAction.java  |   48 -
 .../service/action/AdministrationAction.java    |   38 -
 .../service/action/AuthorizeDownloadAction.java |   54 -
 .../service/action/AuthorizeUserAction.java     |  175 -
 .../admin/service/action/CreateUserAction.java  |   53 -
 .../admin/service/action/DeleteUserAction.java  |   68 -
 .../admin/service/action/DisableUserAction.java |   76 -
 .../service/action/DisableUserGroupAction.java  |   69 -
 .../service/action/FindUserByDnAction.java      |   49 -
 .../service/action/FindUserByIdAction.java      |   49 -
 .../admin/service/action/GetActionAction.java   |   41 -
 .../admin/service/action/GetActionsAction.java  |   48 -
 .../admin/service/action/GetPreviousValues.java |   43 -
 .../service/action/GetUserGroupAction.java      |   50 -
 .../admin/service/action/GetUsersAction.java    |   39 -
 .../service/action/HasPendingUserAccounts.java  |   34 -
 .../action/InvalidateUserAccountAction.java     |   58 -
 .../InvalidateUserGroupAccountsAction.java      |   45 -
 .../service/action/PurgeActionsAction.java      |   51 -
 .../action/RequestUserAccountAction.java        |   67 -
 .../service/action/SeedUserAccountsAction.java  |  164 -
 .../admin/service/action/UngroupUserAction.java |   69 -
 .../service/action/UngroupUserGroupAction.java  |   57 -
 .../admin/service/action/UpdateUserAction.java  |  124 -
 .../UpdateUserAuthoritiesCacheAction.java       |   73 -
 .../service/action/UpdateUserCacheAction.java   |   47 -
 .../service/action/UpdateUserGroupAction.java   |  171 -
 .../service/impl/StandardAuditService.java      |  230 --
 .../admin/service/impl/StandardUserService.java |  663 ----
 .../admin/service/transaction/Transaction.java  |   49 -
 .../service/transaction/TransactionBuilder.java |   25 -
 .../transaction/TransactionException.java       |   40 -
 .../transaction/impl/StandardTransaction.java   |   93 -
 .../impl/StandardTransactionBuilder.java        |   57 -
 .../AuthorityProviderFactoryBean.java           |  529 ---
 ...rdAuthorityProviderConfigurationContext.java |   50 -
 ...dAuthorityProviderInitializationContext.java |   42 -
 .../java/org/apache/nifi/history/History.java   |   56 -
 .../org/apache/nifi/history/HistoryQuery.java   |   99 -
 .../org/apache/nifi/history/PreviousValue.java  |   54 -
 .../org/apache/nifi/user/AccountStatus.java     |   47 -
 .../java/org/apache/nifi/user/NiFiUser.java     |  164 -
 .../org/apache/nifi/user/NiFiUserGroup.java     |   45 -
 .../resources/nifi-administration-context.xml   |   62 -
 .../src/main/xsd/authority-providers.xsd        |   49 -
 .../service/action/AuthorizeUserActionTest.java |  433 ---
 .../service/action/CreateUserActionTest.java    |  144 -
 .../service/action/DisableUserActionTest.java   |  171 -
 .../action/InvalidateUserAccountActionTest.java |  131 -
 .../action/RequestUserAccountActionTest.java    |  127 -
 .../action/SeedUserAccountsActionTest.java      |  263 --
 .../action/SetUserAuthoritiesActionTest.java    |  223 --
 .../framework/client-dto/.gitignore             |    6 -
 .../framework/client-dto/pom.xml                |   46 -
 .../org/apache/nifi/web/api/dto/AboutDTO.java   |   57 -
 .../org/apache/nifi/web/api/dto/BannerDTO.java  |   57 -
 .../nifi/web/api/dto/BulletinBoardDTO.java      |   63 -
 .../apache/nifi/web/api/dto/BulletinDTO.java    |  161 -
 .../nifi/web/api/dto/BulletinQueryDTO.java      |  113 -
 .../org/apache/nifi/web/api/dto/ClusterDTO.java |   61 -
 .../apache/nifi/web/api/dto/ConnectableDTO.java |  146 -
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |  215 --
 .../web/api/dto/ControllerConfigurationDTO.java |  158 -
 .../apache/nifi/web/api/dto/ControllerDTO.java  |  262 --
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   94 -
 .../apache/nifi/web/api/dto/CountersDTO.java    |   61 -
 .../nifi/web/api/dto/DocumentedTypeDTO.java     |   71 -
 .../apache/nifi/web/api/dto/FlowSnippetDTO.java |  141 -
 .../org/apache/nifi/web/api/dto/FunnelDTO.java  |   29 -
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   94 -
 .../nifi/web/api/dto/NiFiComponentDTO.java      |   95 -
 .../org/apache/nifi/web/api/dto/NodeDTO.java    |  188 -
 .../apache/nifi/web/api/dto/NodeEventDTO.java   |   74 -
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |   56 -
 .../org/apache/nifi/web/api/dto/PortDTO.java    |  161 -
 .../apache/nifi/web/api/dto/PositionDTO.java    |   65 -
 .../nifi/web/api/dto/PreviousValueDTO.java      |   73 -
 .../nifi/web/api/dto/ProcessGroupDTO.java       |  219 --
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |  486 ---
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  181 -
 .../nifi/web/api/dto/ProcessorHistoryDTO.java   |   56 -
 .../nifi/web/api/dto/PropertyHistoryDTO.java    |   43 -
 .../nifi/web/api/dto/RelationshipDTO.java       |   69 -
 .../api/dto/RemoteProcessGroupContentsDTO.java  |   56 -
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  279 --
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |  192 -
 .../apache/nifi/web/api/dto/RevisionDTO.java    |   63 -
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |  239 --
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  461 ---
 .../apache/nifi/web/api/dto/TemplateDTO.java    |  117 -
 .../org/apache/nifi/web/api/dto/UserDTO.java    |  177 -
 .../apache/nifi/web/api/dto/UserGroupDTO.java   |   84 -
 .../nifi/web/api/dto/action/ActionDTO.java      |  176 -
 .../nifi/web/api/dto/action/HistoryDTO.java     |   74 -
 .../web/api/dto/action/HistoryQueryDTO.java     |  144 -
 .../component/details/ComponentDetailsDTO.java  |   32 -
 .../component/details/ProcessorDetailsDTO.java  |   41 -
 .../details/RemoteProcessGroupDetailsDTO.java   |   41 -
 .../dto/action/details/ActionDetailsDTO.java    |   34 -
 .../dto/action/details/ConfigureDetailsDTO.java |   69 -
 .../dto/action/details/ConnectDetailsDTO.java   |  126 -
 .../api/dto/action/details/MoveDetailsDTO.java  |   83 -
 .../api/dto/action/details/PurgeDetailsDTO.java |   45 -
 .../web/api/dto/provenance/AttributeDTO.java    |   69 -
 .../web/api/dto/provenance/ProvenanceDTO.java   |  165 -
 .../api/dto/provenance/ProvenanceEventDTO.java  |  630 ---
 .../dto/provenance/ProvenanceOptionsDTO.java    |   43 -
 .../dto/provenance/ProvenanceRequestDTO.java    |  118 -
 .../dto/provenance/ProvenanceResultsDTO.java    |  136 -
 .../ProvenanceSearchableFieldDTO.java           |   84 -
 .../api/dto/provenance/lineage/LineageDTO.java  |  161 -
 .../provenance/lineage/LineageRequestDTO.java   |   88 -
 .../provenance/lineage/LineageResultsDTO.java   |   73 -
 .../provenance/lineage/ProvenanceLinkDTO.java   |  101 -
 .../provenance/lineage/ProvenanceNodeDTO.java   |  162 -
 .../dto/search/ComponentSearchResultDTO.java    |   85 -
 .../web/api/dto/search/NodeSearchResultDTO.java |   56 -
 .../web/api/dto/search/SearchResultsDTO.java    |  128 -
 .../dto/search/UserGroupSearchResultDTO.java    |   42 -
 .../web/api/dto/search/UserSearchResultDTO.java |   56 -
 .../dto/status/ClusterConnectionStatusDTO.java  |   89 -
 .../api/dto/status/ClusterPortStatusDTO.java    |   89 -
 .../dto/status/ClusterProcessorStatusDTO.java   |  117 -
 .../ClusterRemoteProcessGroupStatusDTO.java     |   89 -
 .../web/api/dto/status/ClusterStatusDTO.java    |   44 -
 .../api/dto/status/ClusterStatusHistoryDTO.java |   75 -
 .../web/api/dto/status/ConnectionStatusDTO.java |  198 -
 .../web/api/dto/status/ControllerStatusDTO.java |  187 -
 .../api/dto/status/NodeConnectionStatusDTO.java |   57 -
 .../web/api/dto/status/NodePortStatusDTO.java   |   57 -
 .../api/dto/status/NodeProcessorStatusDTO.java  |   57 -
 .../status/NodeRemoteProcessGroupStatusDTO.java |   57 -
 .../nifi/web/api/dto/status/NodeStatusDTO.java  |   57 -
 .../api/dto/status/NodeStatusHistoryDTO.java    |   57 -
 .../nifi/web/api/dto/status/PortStatusDTO.java  |  142 -
 .../api/dto/status/ProcessGroupStatusDTO.java   |  244 --
 .../web/api/dto/status/ProcessorStatusDTO.java  |  203 -
 .../web/api/dto/status/RemotePortStatusDTO.java |   98 -
 .../dto/status/RemoteProcessGroupStatusDTO.java |  159 -
 .../nifi/web/api/dto/status/StatusDTO.java      |   43 -
 .../web/api/dto/status/StatusDescriptorDTO.java |  101 -
 .../web/api/dto/status/StatusHistoryDTO.java    |   92 -
 .../api/dto/status/StatusHistoryDetailDTO.java  |   56 -
 .../web/api/dto/status/StatusSnapshotDTO.java   |   58 -
 .../nifi/web/api/dto/util/DateTimeAdapter.java  |   47 -
 .../nifi/web/api/dto/util/TimeAdapter.java      |   47 -
 .../nifi/web/api/dto/util/TimestampAdapter.java |   47 -
 .../apache/nifi/web/api/entity/AboutEntity.java |   45 -
 .../nifi/web/api/entity/ActionEntity.java       |   45 -
 .../nifi/web/api/entity/AuthorityEntity.java    |   60 -
 .../nifi/web/api/entity/BannerEntity.java       |   46 -
 .../web/api/entity/BulletinBoardEntity.java     |   45 -
 .../entity/ClusterConnectionStatusEntity.java   |   45 -
 .../nifi/web/api/entity/ClusterEntity.java      |   45 -
 .../web/api/entity/ClusterPortStatusEntity.java |   45 -
 .../entity/ClusterProcessorStatusEntity.java    |   45 -
 .../ClusterRemoteProcessGroupStatusEntity.java  |   45 -
 .../api/entity/ClusterSearchResultsEntity.java  |   46 -
 .../web/api/entity/ClusterStatusEntity.java     |   45 -
 .../api/entity/ClusterStatusHistoryEntity.java  |   45 -
 .../nifi/web/api/entity/ConnectionEntity.java   |   45 -
 .../nifi/web/api/entity/ConnectionsEntity.java  |   47 -
 .../entity/ControllerConfigurationEntity.java   |   45 -
 .../nifi/web/api/entity/ControllerEntity.java   |   45 -
 .../web/api/entity/ControllerStatusEntity.java  |   45 -
 .../nifi/web/api/entity/CounterEntity.java      |   45 -
 .../nifi/web/api/entity/CountersEntity.java     |   46 -
 .../org/apache/nifi/web/api/entity/Entity.java  |   43 -
 .../nifi/web/api/entity/FlowSnippetEntity.java  |   45 -
 .../nifi/web/api/entity/FunnelEntity.java       |   45 -
 .../nifi/web/api/entity/FunnelsEntity.java      |   47 -
 .../nifi/web/api/entity/HistoryEntity.java      |   45 -
 .../nifi/web/api/entity/InputPortEntity.java    |   45 -
 .../nifi/web/api/entity/InputPortsEntity.java   |   47 -
 .../apache/nifi/web/api/entity/LabelEntity.java |   45 -
 .../nifi/web/api/entity/LabelsEntity.java       |   47 -
 .../nifi/web/api/entity/LineageEntity.java      |   45 -
 .../apache/nifi/web/api/entity/NodeEntity.java  |   45 -
 .../nifi/web/api/entity/NodeStatusEntity.java   |   45 -
 .../api/entity/NodeSystemDiagnosticsEntity.java |   45 -
 .../nifi/web/api/entity/OutputPortEntity.java   |   45 -
 .../nifi/web/api/entity/OutputPortsEntity.java  |   47 -
 .../web/api/entity/PrioritizerTypesEntity.java  |   46 -
 .../nifi/web/api/entity/ProcessGroupEntity.java |   45 -
 .../api/entity/ProcessGroupStatusEntity.java    |   45 -
 .../web/api/entity/ProcessGroupsEntity.java     |   46 -
 .../nifi/web/api/entity/ProcessorEntity.java    |   45 -
 .../web/api/entity/ProcessorHistoryEntity.java  |   45 -
 .../web/api/entity/ProcessorTypesEntity.java    |   46 -
 .../nifi/web/api/entity/ProcessorsEntity.java   |   47 -
 .../nifi/web/api/entity/ProvenanceEntity.java   |   40 -
 .../web/api/entity/ProvenanceEventEntity.java   |   45 -
 .../web/api/entity/ProvenanceOptionsEntity.java |   46 -
 .../api/entity/RemoteProcessGroupEntity.java    |   45 -
 .../entity/RemoteProcessGroupPortEntity.java    |   45 -
 .../api/entity/RemoteProcessGroupsEntity.java   |   47 -
 .../web/api/entity/SearchResultsEntity.java     |   46 -
 .../nifi/web/api/entity/SnippetEntity.java      |   45 -
 .../web/api/entity/StatusHistoryEntity.java     |   45 -
 .../web/api/entity/SystemDiagnosticsEntity.java |   45 -
 .../nifi/web/api/entity/TemplateEntity.java     |   45 -
 .../nifi/web/api/entity/TemplatesEntity.java    |   63 -
 .../apache/nifi/web/api/entity/UserEntity.java  |   45 -
 .../nifi/web/api/entity/UserGroupEntity.java    |   45 -
 .../web/api/entity/UserSearchResultsEntity.java |   61 -
 .../apache/nifi/web/api/entity/UsersEntity.java |   64 -
 .../cluster-authorization-provider/.gitignore   |    1 -
 .../cluster-authorization-provider/pom.xml      |   48 -
 .../ClusterManagerAuthorizationProvider.java    |  225 --
 .../NodeAuthorizationProvider.java              |  389 --
 .../protocol/message/DoesDnExistMessage.java    |   56 -
 .../protocol/message/GetAuthoritiesMessage.java |   58 -
 .../message/GetGroupForUserMessage.java         |   55 -
 .../protocol/message/ProtocolMessage.java       |   57 -
 .../message/jaxb/JaxbProtocolUtils.java         |   42 -
 .../protocol/message/jaxb/ObjectFactory.java    |   45 -
 ....apache.nifi.authorization.AuthorityProvider |   16 -
 .../framework/cluster-protocol/.gitignore       |    1 -
 .../framework/cluster-protocol/pom.xml          |   69 -
 .../protocol/ClusterManagerProtocolSender.java  |   69 -
 .../cluster/protocol/ConnectionRequest.java     |   44 -
 .../cluster/protocol/ConnectionResponse.java    |  141 -
 .../apache/nifi/cluster/protocol/Heartbeat.java |   68 -
 .../nifi/cluster/protocol/NodeBulletins.java    |   44 -
 .../nifi/cluster/protocol/NodeIdentifier.java   |  172 -
 .../cluster/protocol/NodeProtocolSender.java    |   73 -
 .../nifi/cluster/protocol/ProtocolContext.java  |   39 -
 .../cluster/protocol/ProtocolException.java     |   40 -
 .../nifi/cluster/protocol/ProtocolHandler.java  |   44 -
 .../nifi/cluster/protocol/ProtocolListener.java |   72 -
 .../protocol/ProtocolMessageMarshaller.java     |   38 -
 .../protocol/ProtocolMessageUnmarshaller.java   |   38 -
 .../nifi/cluster/protocol/StandardDataFlow.java |  105 -
 .../UnknownServiceAddressException.java         |   39 -
 .../impl/ClusterManagerProtocolSenderImpl.java  |  245 --
 .../ClusterManagerProtocolSenderListener.java   |  118 -
 .../protocol/impl/ClusterServiceDiscovery.java  |  181 -
 .../protocol/impl/ClusterServiceLocator.java    |  229 --
 .../impl/ClusterServicesBroadcaster.java        |  182 -
 .../protocol/impl/CopyingInputStream.java       |   77 -
 .../impl/MulticastProtocolListener.java         |  204 -
 .../protocol/impl/NodeProtocolSenderImpl.java   |  171 -
 .../impl/NodeProtocolSenderListener.java        |  115 -
 .../protocol/impl/SocketProtocolListener.java   |  205 -
 .../protocol/jaxb/JaxbProtocolContext.java      |  148 -
 .../jaxb/message/AdaptedConnectionRequest.java  |   40 -
 .../jaxb/message/AdaptedConnectionResponse.java |  109 -
 .../protocol/jaxb/message/AdaptedCounter.java   |   56 -
 .../protocol/jaxb/message/AdaptedDataFlow.java  |   64 -
 .../protocol/jaxb/message/AdaptedHeartbeat.java |   66 -
 .../jaxb/message/AdaptedNodeBulletins.java      |   50 -
 .../jaxb/message/AdaptedNodeIdentifier.java     |   76 -
 .../jaxb/message/ConnectionRequestAdapter.java  |   41 -
 .../jaxb/message/ConnectionResponseAdapter.java |   55 -
 .../protocol/jaxb/message/DataFlowAdapter.java  |   50 -
 .../protocol/jaxb/message/HeartbeatAdapter.java |   54 -
 .../jaxb/message/JaxbProtocolUtils.java         |   42 -
 .../jaxb/message/NodeBulletinsAdapter.java      |   48 -
 .../jaxb/message/NodeIdentifierAdapter.java     |   51 -
 .../protocol/jaxb/message/ObjectFactory.java    |  104 -
 .../message/ConnectionRequestMessage.java       |   46 -
 .../message/ConnectionResponseMessage.java      |   66 -
 .../ControllerStartupFailureMessage.java        |   49 -
 .../protocol/message/DisconnectMessage.java     |   55 -
 .../protocol/message/ExceptionMessage.java      |   44 -
 .../protocol/message/FlowRequestMessage.java    |   46 -
 .../protocol/message/FlowResponseMessage.java   |   44 -
 .../protocol/message/HeartbeatMessage.java      |   43 -
 .../message/MulticastProtocolMessage.java       |   66 -
 .../protocol/message/NodeBulletinsMessage.java  |   43 -
 .../cluster/protocol/message/PingMessage.java   |   55 -
 .../message/PrimaryRoleAssignmentMessage.java   |   56 -
 .../protocol/message/ProtocolMessage.java       |   61 -
 .../message/ReconnectionFailureMessage.java     |   45 -
 .../message/ReconnectionRequestMessage.java     |   94 -
 .../message/ReconnectionResponseMessage.java    |   32 -
 .../message/ServiceBroadcastMessage.java        |   64 -
 .../MulticastConfigurationFactoryBean.java      |   60 -
 .../ServerSocketConfigurationFactoryBean.java   |   65 -
 .../spring/SocketConfigurationFactoryBean.java  |   66 -
 .../resources/nifi-cluster-protocol-context.xml |  110 -
 .../ClusterManagerProtocolSenderImplTest.java   |  134 -
 .../impl/ClusterServiceDiscoveryTest.java       |  135 -
 .../impl/ClusterServiceLocatorTest.java         |  121 -
 .../impl/ClusterServicesBroadcasterTest.java    |  133 -
 .../impl/MulticastProtocolListenerTest.java     |  171 -
 .../impl/NodeProtocolSenderImplTest.java        |  203 -
 .../impl/testutils/DelayedProtocolHandler.java  |   57 -
 .../testutils/ReflexiveProtocolHandler.java     |   47 -
 .../framework/cluster-web/.gitignore            |    1 -
 .../framework/cluster-web/pom.xml               |   50 -
 .../nifi/cluster/context/ClusterContext.java    |   59 -
 .../cluster/context/ClusterContextImpl.java     |   69 -
 .../context/ClusterContextThreadLocal.java      |   47 -
 .../ClusterAwareOptimisticLockingManager.java   |   96 -
 .../framework/cluster/.gitignore                |    1 -
 .../framework-bundle/framework/cluster/pom.xml  |  132 -
 .../cluster/client/MulticastTestClient.java     |  151 -
 .../org/apache/nifi/cluster/event/Event.java    |  122 -
 .../apache/nifi/cluster/event/EventManager.java |   65 -
 .../cluster/event/impl/EventManagerImpl.java    |  143 -
 .../cluster/firewall/ClusterNodeFirewall.java   |   35 -
 .../impl/FileBasedClusterNodeFirewall.java      |  207 -
 .../nifi/cluster/flow/ClusterDataFlow.java      |   45 -
 .../apache/nifi/cluster/flow/DaoException.java  |   40 -
 .../apache/nifi/cluster/flow/DataFlowDao.java   |   62 -
 .../cluster/flow/DataFlowManagementService.java |  115 -
 .../nifi/cluster/flow/PersistedFlowState.java   |   37 -
 .../nifi/cluster/flow/StaleFlowException.java   |   42 -
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |  600 ---
 .../impl/DataFlowManagementServiceImpl.java     |  356 --
 .../nifi/cluster/manager/ClusterManager.java    |  225 --
 .../cluster/manager/HttpClusterManager.java     |  169 -
 .../cluster/manager/HttpRequestReplicator.java  |   99 -
 .../cluster/manager/HttpResponseMapper.java     |   42 -
 .../nifi/cluster/manager/NodeResponse.java      |  329 --
 .../exception/BlockedByFirewallException.java   |   60 -
 .../manager/exception/ClusterException.java     |   40 -
 .../ConnectingNodeMutableRequestException.java  |   41 -
 ...DisconnectedNodeMutableRequestException.java |   41 -
 .../exception/IllegalClusterStateException.java |   41 -
 .../exception/IllegalNodeDeletionException.java |   41 -
 .../IllegalNodeDisconnectionException.java      |   42 -
 .../IllegalNodeReconnectionException.java       |   41 -
 .../IneligiblePrimaryNodeException.java         |   41 -
 .../exception/MutableRequestException.java      |   42 -
 .../exception/NoConnectedNodesException.java    |   41 -
 .../exception/NoResponseFromNodesException.java |   42 -
 .../exception/NodeDisconnectionException.java   |   41 -
 .../exception/NodeReconnectionException.java    |   40 -
 .../PrimaryRoleAssignmentException.java         |   41 -
 .../SafeModeMutableRequestException.java        |   41 -
 .../manager/exception/UnknownNodeException.java |   41 -
 .../exception/UriConstructionException.java     |   42 -
 .../manager/impl/ClusteredEventAccess.java      |  135 -
 .../manager/impl/ClusteredReportingContext.java |  165 -
 .../manager/impl/HttpRequestReplicatorImpl.java |  531 ---
 .../manager/impl/HttpResponseMapperImpl.java    |   85 -
 .../cluster/manager/impl/WebClusterManager.java | 3620 ------------------
 .../java/org/apache/nifi/cluster/node/Node.java |  252 --
 ...anagerProtocolServiceLocatorFactoryBean.java |  116 -
 ...FileBasedClusterNodeFirewallFactoryBean.java |   58 -
 .../spring/WebClusterManagerFactoryBean.java    |  139 -
 .../reporting/ClusteredReportingTaskNode.java   |   49 -
 .../resources/nifi-cluster-manager-context.xml  |  124 -
 .../event/impl/EventManagerImplTest.java        |  119 -
 .../impl/FileBasedClusterNodeFirewallTest.java  |   98 -
 .../impl/DataFlowManagementServiceImplTest.java |  343 --
 .../impl/HttpRequestReplicatorImplTest.java     |  368 --
 .../impl/HttpResponseMapperImplTest.java        |  126 -
 .../manager/impl/TestWebClusterManager.java     |   54 -
 .../cluster/manager/testutils/HttpRequest.java  |  239 --
 .../cluster/manager/testutils/HttpResponse.java |   93 -
 .../manager/testutils/HttpResponseAction.java   |   60 -
 .../cluster/manager/testutils/HttpServer.java   |  240 --
 .../ClusterManagerProtocolSenderImplTest.java   |  133 -
 .../impl/ClusterServiceLocatorTest.java         |  119 -
 .../impl/ClusterServicesBroadcasterTest.java    |  131 -
 .../impl/MulticastProtocolListenerTest.java     |  171 -
 .../impl/NodeProtocolSenderImplTest.java        |  201 -
 .../impl/SocketProtocolListenerTest.java        |  132 -
 .../testutils/DelayedProtocolHandler.java       |   57 -
 .../testutils/ReflexiveProtocolHandler.java     |   47 -
 .../cluster/src/test/resources/logback-test.xml |   48 -
 .../apache/nifi/cluster/firewall/impl/empty.txt |    0
 .../apache/nifi/cluster/firewall/impl/ips.txt   |   12 -
 .../framework/core-api/.gitignore               |    1 -
 .../framework-bundle/framework/core-api/pom.xml |   56 -
 .../nifi/cluster/AdaptedNodeInformation.java    |   66 -
 .../nifi/cluster/ClusterNodeInformation.java    |   67 -
 .../org/apache/nifi/cluster/NodeInformant.java  |   22 -
 .../apache/nifi/cluster/NodeInformation.java    |   98 -
 .../nifi/cluster/NodeInformationAdapter.java    |   39 -
 .../apache/nifi/cluster/protocol/DataFlow.java  |   41 -
 .../apache/nifi/connectable/Connectable.java    |  291 --
 .../nifi/connectable/ConnectableType.java       |   44 -
 .../org/apache/nifi/connectable/Connection.java |   78 -
 .../org/apache/nifi/connectable/Funnel.java     |   24 -
 .../java/org/apache/nifi/connectable/Port.java  |   31 -
 .../org/apache/nifi/connectable/Position.java   |   36 -
 .../java/org/apache/nifi/connectable/Size.java  |   36 -
 .../controller/AbstractConfiguredComponent.java |  280 --
 .../apache/nifi/controller/AbstractPort.java    |  636 ---
 .../apache/nifi/controller/Availability.java    |   24 -
 .../nifi/controller/ConfiguredComponent.java    |   63 -
 .../nifi/controller/ContentAvailability.java    |   65 -
 .../org/apache/nifi/controller/Counter.java     |   32 -
 .../nifi/controller/EventBasedWorker.java       |   32 -
 .../org/apache/nifi/controller/Heartbeater.java |   22 -
 .../nifi/controller/ProcessScheduler.java       |  146 -
 .../apache/nifi/controller/ProcessorNode.java   |   80 -
 .../nifi/controller/ReportingTaskNode.java      |   56 -
 .../nifi/controller/StandardFlowFileQueue.java  | 1096 ------
 .../apache/nifi/controller/StandardFunnel.java  |  541 ---
 .../controller/ValidationContextFactory.java    |   27 -
 .../org/apache/nifi/controller/WorkerQueue.java |   36 -
 .../exception/CommunicationsException.java      |   40 -
 ...ControllerServiceAlreadyExistsException.java |   30 -
 .../ControllerServiceNotFoundException.java     |   51 -
 .../ProcessorInstantiationException.java        |   27 -
 .../exception/ProcessorLifeCycleException.java  |   30 -
 .../org/apache/nifi/controller/label/Label.java |   48 -
 .../ReportingTaskInstantiationException.java    |   31 -
 .../repository/ContentNotFoundException.java    |   48 -
 .../repository/CounterRepository.java           |   34 -
 .../controller/repository/FlowFileEvent.java    |   54 -
 .../repository/FlowFileEventRepository.java     |   50 -
 .../repository/RepositoryStatusReport.java      |   28 -
 .../service/ControllerServiceNode.java          |   40 -
 .../service/ControllerServiceProvider.java      |   47 -
 .../service/ControllerServiceReference.java     |   50 -
 .../org/apache/nifi/events/BulletinFactory.java |   52 -
 .../nifi/events/BulletinProcessingStrategy.java |   27 -
 .../apache/nifi/events/ComponentBulletin.java   |   30 -
 .../org/apache/nifi/events/SystemBulletin.java  |   30 -
 .../org/apache/nifi/groups/ProcessGroup.java    |  723 ----
 .../apache/nifi/groups/ProcessGroupCounts.java  |   66 -
 .../apache/nifi/groups/RemoteProcessGroup.java  |  255 --
 .../RemoteProcessGroupPortDescriptor.java       |   92 -
 .../org/apache/nifi/logging/LogMessage.java     |   74 -
 .../org/apache/nifi/logging/LogObserver.java    |   22 -
 .../org/apache/nifi/logging/LogRepository.java  |   67 -
 .../nifi/logging/LogRepositoryFactory.java      |   61 -
 .../java/org/apache/nifi/nar/NarCloseable.java  |   44 -
 .../nifi/nar/NarThreadContextClassLoader.java   |  188 -
 .../main/java/org/apache/nifi/remote/Peer.java  |  107 -
 .../java/org/apache/nifi/remote/PeerStatus.java |   72 -
 .../nifi/remote/PortAuthorizationResult.java    |   25 -
 .../nifi/remote/RemoteAuthorizationState.java   |   27 -
 .../org/apache/nifi/remote/RemoteGroupPort.java |   35 -
 .../org/apache/nifi/remote/RootGroupPort.java   |   78 -
 .../apache/nifi/remote/TransferDirection.java   |   23 -
 .../nifi/remote/VersionedRemoteResource.java    |   24 -
 .../apache/nifi/remote/codec/FlowFileCodec.java |   79 -
 .../remote/exception/BadRequestException.java   |   30 -
 .../remote/exception/HandshakeException.java    |   30 -
 .../exception/NotAuthorizedException.java       |   26 -
 .../exception/PortNotRunningException.java      |   26 -
 .../remote/exception/ProtocolException.java     |   34 -
 .../exception/RequestExpiredException.java      |   26 -
 .../remote/exception/UnknownPortException.java  |   26 -
 .../nifi/remote/protocol/ClientProtocol.java    |   78 -
 .../remote/protocol/CommunicationsInput.java    |   27 -
 .../remote/protocol/CommunicationsOutput.java   |   27 -
 .../remote/protocol/CommunicationsSession.java  |   64 -
 .../nifi/remote/protocol/RequestType.java       |   43 -
 .../nifi/remote/protocol/ServerProtocol.java    |  143 -
 .../framework-bundle/framework/core/.gitignore  |    1 -
 .../framework-bundle/framework/core/pom.xml     |  121 -
 .../apache/nifi/cluster/BulletinsPayload.java   |   95 -
 .../nifi/cluster/ConnectionException.java       |   42 -
 .../nifi/cluster/DisconnectionException.java    |   42 -
 .../apache/nifi/cluster/HeartbeatPayload.java   |  170 -
 .../org/apache/nifi/connectable/LocalPort.java  |  172 -
 .../nifi/connectable/StandardConnection.java    |  336 --
 .../nifi/controller/EventDrivenWorkerQueue.java |  329 --
 .../nifi/controller/FileSystemSwapManager.java  |  768 ----
 .../apache/nifi/controller/FlowController.java  | 3579 -----------------
 .../nifi/controller/FlowFromDOMFactory.java     |  418 --
 .../controller/FlowSerializationException.java  |   48 -
 .../apache/nifi/controller/FlowSerializer.java  |   42 -
 .../FlowSynchronizationException.java           |   47 -
 .../nifi/controller/FlowSynchronizer.java       |   53 -
 .../nifi/controller/FlowUnmarshaller.java       |   78 -
 .../apache/nifi/controller/SnippetManager.java  |   96 -
 .../apache/nifi/controller/StandardCounter.java |  108 -
 .../nifi/controller/StandardFlowSerializer.java |  404 --
 .../nifi/controller/StandardFlowService.java    |  875 -----
 .../controller/StandardFlowSynchronizer.java    | 1026 -----
 .../nifi/controller/StandardProcessorNode.java  | 1243 ------
 .../apache/nifi/controller/StandardSnippet.java |  186 -
 .../org/apache/nifi/controller/Template.java    |   37 -
 .../apache/nifi/controller/TemplateManager.java |  507 ---
 .../controller/UninheritableFlowException.java  |   48 -
 .../exception/FlowFileConsumptionException.java |   38 -
 .../ReportingTaskInstantiationException.java    |   31 -
 .../exception/ValidationException.java          |   47 -
 .../nifi/controller/label/StandardLabel.java    |  110 -
 .../reporting/AbstractReportingTaskNode.java    |  111 -
 .../reporting/StandardReportingContext.java     |  132 -
 .../StandardReportingInitializationContext.java |   96 -
 .../reporting/StandardReportingTaskNode.java    |   40 -
 .../repository/BatchingSessionFactory.java      |  242 --
 .../repository/ConnectionSwapInfo.java          |   58 -
 .../repository/FileSystemRepository.java        | 1351 -------
 .../controller/repository/ProcessContext.java   |  291 --
 .../repository/ProvenanceEventEnricher.java     |   34 -
 .../repository/RepositoryPurgeException.java    |   59 -
 .../repository/RingBufferEventRepository.java   |  312 --
 .../repository/StandardCounterRepository.java   |  109 -
 .../repository/StandardFlowFileEvent.java       |  237 --
 .../repository/StandardFlowFileRecord.java      |  312 --
 .../repository/StandardProcessSession.java      | 2684 -------------
 .../StandardProcessSessionFactory.java          |   33 -
 .../repository/StandardProvenanceReporter.java  |  431 ---
 .../repository/StandardRepositoryRecord.java    |  196 -
 .../StandardRepositoryStatusReport.java         |   88 -
 .../repository/VolatileContentRepository.java   |  647 ----
 .../repository/VolatileFlowFileRepository.java  |  109 -
 .../WriteAheadFlowFileRepository.java           |  864 -----
 .../repository/claim/ContentDirection.java      |   35 -
 .../repository/claim/StandardContentClaim.java  |  154 -
 .../claim/StandardContentClaimManager.java      |  155 -
 .../repository/io/ArrayManagedOutputStream.java |  245 --
 .../repository/io/ByteCountingInputStream.java  |  100 -
 .../repository/io/ByteCountingOutputStream.java |   64 -
 .../io/DisableOnCloseOutputStream.java          |   73 -
 .../io/FlowFileAccessInputStream.java           |  168 -
 .../io/FlowFileAccessOutputStream.java          |   95 -
 .../repository/io/LimitedInputStream.java       |  107 -
 .../controller/repository/io/LongHolder.java    |   48 -
 .../controller/repository/io/MemoryManager.java |   55 -
 .../repository/io/SyncOnCloseOutputStream.java  |   58 -
 .../scheduling/ConnectableProcessContext.java   |  194 -
 .../scheduling/EventDrivenSchedulingAgent.java  |  346 --
 .../scheduling/ProcessContextFactory.java       |   51 -
 .../scheduling/QuartzSchedulingAgent.java       |  220 --
 .../controller/scheduling/ScheduleState.java    |   90 -
 .../controller/scheduling/SchedulingAgent.java  |   45 -
 .../scheduling/StandardProcessScheduler.java    |  569 ---
 .../scheduling/TimerDrivenSchedulingAgent.java  |  139 -
 .../service/ControllerServiceLoader.java        |  156 -
 .../service/StandardConfigurationContext.java   |   64 -
 ...dControllerServiceInitializationContext.java |   64 -
 .../service/StandardControllerServiceNode.java  |  125 -
 .../StandardControllerServiceProvider.java      |  196 -
 .../StandardControllerServiceReference.java     |  100 -
 .../history/StandardMetricDescriptor.java       |  128 -
 .../status/history/StandardStatusHistory.java   |   54 -
 .../status/history/StandardStatusSnapshot.java  |   70 -
 .../status/history/StatusHistoryUtil.java       |  107 -
 .../VolatileComponentStatusRepository.java      |  649 ----
 .../tasks/ContinuallyRunConnectableTask.java    |   97 -
 .../tasks/ContinuallyRunProcessorTask.java      |  185 -
 .../nifi/controller/tasks/ExpireFlowFiles.java  |  114 -
 .../controller/tasks/ReportingTaskWrapper.java  |   63 -
 .../nifi/diagnostics/DiagnosticUtils.java       |   28 -
 .../nifi/diagnostics/GarbageCollection.java     |   60 -
 .../apache/nifi/diagnostics/StorageUsage.java   |   72 -
 .../nifi/diagnostics/SystemDiagnostics.java     |  212 -
 .../diagnostics/SystemDiagnosticsFactory.java   |  133 -
 .../nifi/encrypt/EncryptionException.java       |   43 -
 .../apache/nifi/encrypt/StringEncryptor.java    |  152 -
 .../java/org/apache/nifi/engine/FlowEngine.java |  135 -
 .../events/NodeBulletinProcessingStrategy.java  |   66 -
 .../nifi/events/VolatileBulletinRepository.java |  240 --
 .../events/network/CommunicationsFailure.java   |   36 -
 .../nifi/events/network/NetworkTransfer.java    |   48 -
 .../nifi/fingerprint/FingerprintException.java  |   41 -
 .../nifi/fingerprint/FingerprintFactory.java    |  985 -----
 .../nifi/groups/StandardProcessGroup.java       | 2019 ----------
 .../org/apache/nifi/jaxb/AdaptedBulletin.java   |  100 -
 .../org/apache/nifi/jaxb/AdaptedCounter.java    |   64 -
 .../org/apache/nifi/jaxb/BulletinAdapter.java   |   59 -
 .../org/apache/nifi/jaxb/CounterAdapter.java    |   51 -
 .../org/apache/nifi/lifecycle/LifeCycle.java    |   54 -
 .../nifi/lifecycle/LifeCycleException.java      |   48 -
 .../nifi/lifecycle/LifeCycleStartException.java |   49 -
 .../nifi/lifecycle/LifeCycleStopException.java  |   48 -
 .../nifi/logging/ProcessorLogObserver.java      |   48 -
 .../repository/StandardLogRepository.java       |  173 -
 .../nifi/persistence/FlowConfigurationDAO.java  |  135 -
 .../StandardSnippetDeserializer.java            |   42 -
 .../persistence/StandardSnippetSerializer.java  |   47 -
 .../StandardXMLFlowConfigurationDAO.java        |  345 --
 .../nifi/persistence/TemplateDeserializer.java  |   42 -
 .../nifi/persistence/TemplateSerializer.java    |   47 -
 .../nifi/processor/SimpleProcessLogger.java     |  307 --
 .../nifi/processor/StandardProcessContext.java  |  173 -
 .../StandardProcessorInitializationContext.java |   49 -
 .../nifi/processor/StandardPropertyValue.java   |  163 -
 .../processor/StandardSchedulingContext.java    |  113 -
 .../processor/StandardValidationContext.java    |   97 -
 .../StandardValidationContextFactory.java       |   39 -
 .../nifi/remote/StandardRemoteProcessGroup.java | 1621 --------
 ...tandardRemoteProcessGroupPortDescriptor.java |  148 -
 .../org/apache/nifi/services/FlowService.java   |  139 -
 .../nifi/spring/FlowControllerFactoryBean.java  |  103 -
 .../spring/RingBufferEventRepositoryBean.java   |   45 -
 .../spring/StandardFlowServiceFactoryBean.java  |   98 -
 .../apache/nifi/util/ComponentStatusReport.java |  139 -
 .../java/org/apache/nifi/util/Connectables.java |   56 -
 .../java/org/apache/nifi/util/DomUtils.java     |   79 -
 .../org/apache/nifi/util/ReflectionUtils.java   |  157 -
 .../java/org/apache/nifi/util/SnippetUtils.java |  241 --
 .../ControllerServiceConfiguration.xsd          |   61 -
 .../src/main/resources/FlowConfiguration.xsd    |  335 --
 .../resources/ReportingTaskConfiguration.xsd    |   87 -
 .../core/src/main/resources/nifi-context.xml    |   49 -
 .../nifi/cluster/HeartbeatPayloadTest.java      |  121 -
 .../controller/StandardFlowServiceTest.java     |  300 --
 .../controller/TestFileSystemSwapManager.java   |  104 -
 .../repository/TestFileSystemRepository.java    |  352 --
 .../TestRingBufferEventRepository.java          |  138 -
 .../repository/TestStandardProcessSession.java  | 1177 ------
 .../TestStandardProvenanceReporter.java         |   65 -
 .../TestVolatileContentRepository.java          |  183 -
 .../TestWriteAheadFlowFileRepository.java       |  135 -
 .../repository/io/TestLimitedOutputStream.java  |   76 -
 .../controller/repository/util/DiskUtils.java   |   77 -
 .../fingerprint/FingerprintFactoryTest.java     |   60 -
 .../processor/TestStandardPropertyValue.java    |  167 -
 .../processors/DataGeneratorTestProcessor.java  |   85 -
 .../StubAttributeLoggerProcessor.java           |  111 -
 .../StubTerminationFileProcessor.java           |   40 -
 .../core/src/test/resources/conf/0bytes.xml     |    0
 .../test/resources/conf/all-flow-corrupt.xml    |  201 -
 .../resources/conf/all-flow-inheritable.xml     |  196 -
 .../resources/conf/all-flow-uninheritable.xml   |  202 -
 .../core/src/test/resources/conf/all-flow.xml   |  198 -
 .../src/test/resources/conf/nifi.properties     |  127 -
 .../test/resources/conf/only-termination.xml    |   37 -
 .../src/test/resources/conf/remote-flow.xml     |  145 -
 .../src/test/resources/conf/standard-flow.xml   |  196 -
 .../core/src/test/resources/conf/taskConfig.xml |   17 -
 .../test/resources/conf/termination-only.xml    |    0
 .../framework/core/src/test/resources/hello.txt |    1 -
 .../core/src/test/resources/logback-test.xml    |   36 -
 .../test/resources/nifi-with-remote.properties  |  127 -
 .../core/src/test/resources/nifi.properties     |  127 -
 .../test/resources/nifi/fingerprint/flow1a.xml  |  160 -
 .../test/resources/nifi/fingerprint/flow1b.xml  |  160 -
 .../test/resources/nifi/fingerprint/flow2.xml   |  160 -
 .../core/src/test/resources/old-swap-file.swap  |  Bin 1730054 -> 0 bytes
 .../file-authorization-provider/pom.xml         |   81 -
 .../FileAuthorizationProvider.java              |  584 ---
 ....apache.nifi.authorization.AuthorityProvider |   15 -
 .../src/main/xsd/users.xsd                      |   64 -
 .../FileAuthorizationProviderTest.java          |  127 -
 .../framework-bundle/framework/nar/.gitignore   |    1 -
 .../framework-bundle/framework/nar/pom.xml      |   36 -
 .../org/apache/nifi/nar/ExtensionManager.java   |  188 -
 .../org/apache/nifi/nar/ExtensionMapping.java   |   73 -
 .../org/apache/nifi/nar/NarClassLoader.java     |  227 --
 .../org/apache/nifi/nar/NarClassLoaders.java    |  301 --
 .../java/org/apache/nifi/nar/NarUnpacker.java   |  351 --
 .../java/org/apache/nifi/util/FileUtils.java    |  187 -
 nar-bundles/framework-bundle/framework/pom.xml  |   51 -
 .../framework/resources/pom.xml                 |   52 -
 .../src/main/assembly/dependencies.xml          |   36 -
 .../src/main/resources/bin/dump-nifi.bat        |   33 -
 .../src/main/resources/bin/nifi-status.bat      |   33 -
 .../resources/src/main/resources/bin/nifi.sh    |  181 -
 .../src/main/resources/bin/run-nifi.bat         |   33 -
 .../src/main/resources/bin/start-nifi.bat       |   33 -
 .../src/main/resources/bin/stop-nifi.bat        |   33 -
 .../main/resources/conf/authority-providers.xml |   43 -
 .../main/resources/conf/authorized-users.xml    |   57 -
 .../src/main/resources/conf/bootstrap.conf      |   57 -
 .../main/resources/conf/controller-services.xml |   18 -
 .../src/main/resources/conf/logback.xml         |  105 -
 .../src/main/resources/conf/nifi.properties     |  159 -
 .../src/main/resources/conf/reporting-tasks.xml |   49 -
 .../framework/runtime/.gitignore                |    2 -
 .../framework-bundle/framework/runtime/pom.xml  |   42 -
 .../java/org/apache/nifi/BootstrapListener.java |  378 --
 .../src/main/java/org/apache/nifi/NiFi.java     |  223 --
 .../main/java/org/apache/nifi/NiFiServer.java   |   31 -
 .../apache/nifi/util/LimitingInputStream.java   |  107 -
 .../framework/security/.gitignore               |    1 -
 .../framework-bundle/framework/security/pom.xml |   42 -
 .../util/SslContextCreationException.java       |   40 -
 .../security/util/SslContextFactory.java        |  119 -
 .../framework/security/util/SslException.java   |   40 -
 .../security/util/SslServerSocketFactory.java   |   81 -
 ...SslServerSocketFactoryCreationException.java |   40 -
 .../security/util/SslSocketFactory.java         |   91 -
 .../util/SslSocketFactoryCreationException.java |   40 -
 .../security/util/SslContextFactoryTest.java    |   69 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../src/test/resources/log4j.properties         |   21 -
 .../framework/site-to-site/.gitignore           |    1 -
 .../framework/site-to-site/pom.xml              |   74 -
 .../util/RemoteProcessGroupUtils.java           |  216 --
 .../remote/AbstractCommunicationsSession.java   |   54 -
 .../nifi/remote/RemoteResourceFactory.java      |  202 -
 .../nifi/remote/RemoteResourceManager.java      |  227 --
 .../apache/nifi/remote/RemoteSiteListener.java  |   33 -
 .../nifi/remote/SocketRemoteSiteListener.java   |  340 --
 .../nifi/remote/StandardRemoteGroupPort.java    |  764 ----
 .../nifi/remote/StandardRootGroupPort.java      |  629 ---
 .../remote/codec/StandardFlowFileCodec.java     |  169 -
 .../exception/UnsupportedCodecException.java    |   31 -
 .../SocketChannelCommunicationsSession.java     |   90 -
 .../remote/io/socket/SocketChannelInput.java    |   66 -
 .../remote/io/socket/SocketChannelOutput.java   |   58 -
 .../SSLSocketChannelCommunicationsSession.java  |   93 -
 .../io/socket/ssl/SSLSocketChannelInput.java    |   50 -
 .../io/socket/ssl/SSLSocketChannelOutput.java   |   44 -
 .../socket/ClusterManagerServerProtocol.java    |  208 -
 .../protocol/socket/HandshakeProperty.java      |   23 -
 .../nifi/remote/protocol/socket/Response.java   |   51 -
 .../remote/protocol/socket/ResponseCode.java    |  152 -
 .../protocol/socket/SocketClientProtocol.java   |  510 ---
 .../socket/SocketFlowFileServerProtocol.java    |  581 ---
 .../org.apache.nifi.remote.codec.FlowFileCodec  |   15 -
 ...g.apache.nifi.remote.protocol.ClientProtocol |   15 -
 ...g.apache.nifi.remote.protocol.ServerProtocol |   16 -
 .../remote/TestStandardRemoteGroupPort.java     |   97 -
 .../remote/TestStandardSiteToSiteProtocol.java  |  116 -
 .../io/socket/TestSocketChannelStreams.java     |  231 --
 .../io/socket/ssl/TestSSLSocketChannel.java     |  381 --
 .../test/resources/dummy-certs/localhost-ks.jks |  Bin 1298 -> 0 bytes
 .../test/resources/dummy-certs/localhost-ts.jks |  Bin 891 -> 0 bytes
 .../src/test/resources/logback-test.xml         |   30 -
 .../src/test/resources/nifi.properties          |  111 -
 .../framework/user-actions/.gitignore           |    1 -
 .../framework/user-actions/pom.xml              |   27 -
 .../java/org/apache/nifi/action/Action.java     |  121 -
 .../java/org/apache/nifi/action/Component.java  |   31 -
 .../java/org/apache/nifi/action/Operation.java  |   36 -
 .../component/details/ComponentDetails.java     |   26 -
 .../component/details/ProcessorDetails.java     |   34 -
 .../details/RemoteProcessGroupDetails.java      |   34 -
 .../nifi/action/details/ActionDetails.java      |   26 -
 .../nifi/action/details/ConfigureDetails.java   |   52 -
 .../nifi/action/details/ConnectDetails.java     |   90 -
 .../apache/nifi/action/details/MoveDetails.java |   61 -
 .../nifi/action/details/PurgeDetails.java       |   41 -
 .../web/custom-ui-utilities/.gitignore          |    1 -
 .../framework/web/custom-ui-utilities/pom.xml   |   36 -
 .../web/HttpServletRequestContextConfig.java    |  118 -
 .../framework/web/nifi-jetty/.gitignore         |    1 -
 .../framework/web/nifi-jetty/pom.xml            |  166 -
 .../src/main/assembly/distribution.xml          |   42 -
 .../org/apache/nifi/web/server/JettyServer.java |  686 ----
 .../server/ServerConfigurationException.java    |   36 -
 .../org/apache/nifi/web/webdefault.xml          |  556 ---
 .../framework/web/nifi-web-api/.gitignore       |    1 -
 .../framework/web/nifi-web-api/pom.xml          |  305 --
 .../src/main/enunciate/enunciate.xml            |   36 -
 .../src/main/enunciate/images/home.png          |  Bin 144 -> 0 bytes
 .../src/main/enunciate/override.css             |  178 -
 .../org/apache/nifi/aop/MethodProfiler.java     |   45 -
 .../apache/nifi/audit/ControllerAuditor.java    |  257 --
 .../org/apache/nifi/audit/FunnelAuditor.java    |  135 -
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  119 -
 .../java/org/apache/nifi/audit/PortAuditor.java |  334 --
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  263 --
 .../org/apache/nifi/audit/ProcessorAuditor.java |  408 --
 .../apache/nifi/audit/RelationshipAuditor.java  |  412 --
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  376 --
 .../org/apache/nifi/audit/SnippetAuditor.java   |  519 ---
 .../nifi/web/ClusterRequestException.java       |   38 -
 .../IllegalClusterResourceRequestException.java |   33 -
 .../org/apache/nifi/web/NiFiCoreException.java  |   32 -
 .../org/apache/nifi/web/NiFiServiceFacade.java  | 1263 ------
 .../apache/nifi/web/NiFiServiceFacadeLock.java  |  177 -
 .../nifi/web/StandardNiFiServiceFacade.java     | 2877 --------------
 .../apache/nifi/web/StandardNiFiWebContext.java |  336 --
 .../nifi/web/api/ApplicationResource.java       |  399 --
 .../nifi/web/api/BulletinBoardResource.java     |  116 -
 .../apache/nifi/web/api/ClusterResource.java    |  724 ----
 .../apache/nifi/web/api/ConnectionResource.java |  771 ----
 .../apache/nifi/web/api/ControllerResource.java |  803 ----
 .../org/apache/nifi/web/api/FunnelResource.java |  493 ---
 .../apache/nifi/web/api/HistoryResource.java    |  271 --
 .../apache/nifi/web/api/InputPortResource.java  |  531 ---
 .../org/apache/nifi/web/api/LabelResource.java  |  547 ---
 .../org/apache/nifi/web/api/NodeResource.java   |  302 --
 .../apache/nifi/web/api/OutputPortResource.java |  532 ---
 .../nifi/web/api/ProcessGroupResource.java      | 1102 ------
 .../apache/nifi/web/api/ProcessorResource.java  |  693 ----
 .../apache/nifi/web/api/ProvenanceResource.java | 1068 ------
 .../web/api/RemoteProcessGroupResource.java     |  912 -----
 .../apache/nifi/web/api/SnippetResource.java    |  601 ---
 .../nifi/web/api/SystemDiagnosticsResource.java |   81 -
 .../apache/nifi/web/api/TemplateResource.java   |  400 --
 .../apache/nifi/web/api/UserGroupResource.java  |  383 --
 .../org/apache/nifi/web/api/UserResource.java   |  469 ---
 .../api/config/AccessDeniedExceptionMapper.java |   54 -
 .../config/AccountNotFoundExceptionMapper.java  |   47 -
 .../config/AdministrationExceptionMapper.java   |   48 -
 ...ationCredentialsNotFoundExceptionMapper.java |   48 -
 .../web/api/config/ClusterExceptionMapper.java  |   47 -
 ...ectingNodeMutableRequestExceptionMapper.java |   49 -
 ...nectedNodeMutableRequestExceptionMapper.java |   49 -
 .../config/IllegalArgumentExceptionMapper.java  |   46 -
 ...alClusterResourceRequestExceptionMapper.java |   47 -
 .../IllegalClusterStateExceptionMapper.java     |   47 -
 .../IllegalNodeDeletionExceptionMapper.java     |   46 -
 ...IllegalNodeDisconnectionExceptionMapper.java |   46 -
 .../IllegalNodeReconnectionExceptionMapper.java |   46 -
 .../api/config/IllegalStateExceptionMapper.java |   46 -
 .../IneligiblePrimaryNodeExceptionMapper.java   |   47 -
 .../config/InvalidRevisionExceptionMapper.java  |   47 -
 .../api/config/JsonMappingExceptionMapper.java  |   46 -
 .../api/config/JsonParseExceptionMapper.java    |   46 -
 .../config/MutableRequestExceptionMapper.java   |   48 -
 .../web/api/config/NiFiCoreExceptionMapper.java |   47 -
 .../config/NoConnectedNodesExceptionMapper.java |   47 -
 .../NoResponseFromNodesExceptionMapper.java     |   49 -
 .../NodeDisconnectionExceptionMapper.java       |   47 -
 .../config/NodeReconnectionExceptionMapper.java |   47 -
 .../PrimaryRoleAssignmentExceptionMapper.java   |   47 -
 .../config/ResourceNotFoundExceptionMapper.java |   48 -
 .../SafeModeMutableRequestExceptionMapper.java  |   48 -
 .../nifi/web/api/config/ThrowableMapper.java    |   46 -
 .../api/config/UnknownNodeExceptionMapper.java  |   47 -
 .../api/config/ValidationExceptionMapper.java   |   49 -
 .../config/WebApplicationExceptionMapper.java   |   63 -
 .../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 -----------
 .../org/apache/nifi/web/api/package-info.java   |   58 -
 .../request/BulletinBoardPatternParameter.java  |   47 -
 .../nifi/web/api/request/ClientIdParameter.java |   45 -
 .../api/request/ConnectableTypeParameter.java   |   41 -
 .../nifi/web/api/request/DateTimeParameter.java |   63 -
 .../nifi/web/api/request/DoubleParameter.java   |   40 -
 .../nifi/web/api/request/IntegerParameter.java  |   40 -
 .../nifi/web/api/request/LongParameter.java     |   40 -
 .../ApplicationStartupContextListener.java      |  157 -
 .../nifi/web/controller/ControllerFacade.java   | 1392 -------
 .../web/controller/StandardSearchContext.java   |   64 -
 .../org/apache/nifi/web/dao/ConnectionDAO.java  |  112 -
 .../java/org/apache/nifi/web/dao/FunnelDAO.java |   88 -
 .../java/org/apache/nifi/web/dao/LabelDAO.java  |   79 -
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   96 -
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   89 -
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   97 -
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |  135 -
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   92 -
 .../org/apache/nifi/web/dao/TemplateDAO.java    |   77 -
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |   72 -
 .../web/dao/impl/StandardConnectionDAO.java     |  519 ---
 .../nifi/web/dao/impl/StandardFunnelDAO.java    |  180 -
 .../nifi/web/dao/impl/StandardInputPortDAO.java |  309 --
 .../nifi/web/dao/impl/StandardLabelDAO.java     |  184 -
 .../web/dao/impl/StandardOutputPortDAO.java     |  309 --
 .../web/dao/impl/StandardProcessGroupDAO.java   |  171 -
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  503 ---
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  402 --
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |  344 --
 .../nifi/web/dao/impl/StandardTemplateDAO.java  |  141 -
 .../nifi/web/filter/NodeRequestFilter.java      |  130 -
 .../apache/nifi/web/filter/RequestLogger.java   |   78 -
 .../nifi/web/filter/ThreadLocalFilter.java      |   54 -
 .../org/apache/nifi/web/filter/TimerFilter.java |   70 -
 .../nifi/web/util/ClientResponseUtils.java      |   48 -
 .../nifi/web/util/DownloadableContent.java      |   47 -
 .../org/apache/nifi/web/util/SnippetUtils.java  |  426 ---
 .../src/main/resources/nifi-web-api-context.xml |  303 --
 .../src/main/webapp/WEB-INF/web.xml             |  105 -
 .../apache/nifi/integration/NiFiWebApiTest.java |  274 --
 .../accesscontrol/AdminAccessControlTest.java   | 1090 ------
 .../accesscontrol/DfmAccessControlTest.java     | 1410 -------
 .../ReadOnlyAccessControlTest.java              | 1064 -----
 .../util/NiFiTestAuthorizationProvider.java     |  179 -
 .../nifi/integration/util/NiFiTestServer.java   |  235 --
 .../nifi/integration/util/NiFiTestUser.java     |  257 --
 .../integration/util/SourceTestProcessor.java   |   49 -
 .../util/TerminationTestProcessor.java          |   44 -
 .../nifi/integration/util/TestPrioritizer.java  |   31 -
 ....apache.nifi.authorization.AuthorityProvider |   15 -
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   15 -
 .../org.apache.nifi.processor.Processor         |   16 -
 .../access-control/authority-providers.xml      |   24 -
 .../access-control/controller-services.xml      |   18 -
 .../resources/access-control/localhost-ks.jks   |  Bin 2246 -> 0 bytes
 .../resources/access-control/localhost-ts.jks   |  Bin 957 -> 0 bytes
 .../resources/access-control/nifi.properties    |  136 -
 .../access-control/reporting-tasks.xml          |   17 -
 .../src/test/resources/logback-test.xml         |   36 -
 .../framework/web/nifi-web-docs/.gitignore      |    1 -
 .../framework/web/nifi-web-docs/pom.xml         |   57 -
 .../nifi/web/docs/DocumentationController.java  |   91 -
 .../main/webapp/WEB-INF/jsp/documentation.jsp   |  130 -
 .../WEB-INF/jsp/no-documentation-found.jsp      |   31 -
 .../src/main/webapp/WEB-INF/web.xml             |   33 -
 .../src/main/webapp/css/component-usage.css     |  127 -
 .../nifi-web-docs/src/main/webapp/css/main.css  |  214 --
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 189 -> 0 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 1697 -> 0 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 232 -> 0 bytes
 .../src/main/webapp/js/application.js           |  282 --
 .../framework/web/nifi-web-error/.gitignore     |    1 -
 .../framework/web/nifi-web-error/pom.xml        |   48 -
 .../apache/nifi/web/filter/CatchAllFilter.java  |   45 -
 .../src/main/webapp/WEB-INF/web.xml             |   26 -
 .../nifi-web-error/src/main/webapp/index.jsp    |   37 -
 .../framework/web/nifi-web-ui/.gitignore        |    1 -
 .../framework/web/nifi-web-ui/pom.xml           |  655 ----
 .../apache/nifi/web/filter/IeEdgeHeader.java    |   57 -
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 -
 .../filters/bulletin-board-min.properties       |   18 -
 .../resources/filters/bulletin-board.properties |   25 -
 .../resources/filters/canvas-min.properties     |   18 -
 .../main/resources/filters/canvas.properties    |   69 -
 .../resources/filters/cluster-min.properties    |   18 -
 .../main/resources/filters/cluster.properties   |   26 -
 .../resources/filters/counters-min.properties   |   18 -
 .../main/resources/filters/counters.properties  |   26 -
 .../resources/filters/history-min.properties    |   18 -
 .../main/resources/filters/history.properties   |   27 -
 .../resources/filters/provenance-min.properties |   18 -
 .../resources/filters/provenance.properties     |   26 -
 .../resources/filters/summary-min.properties    |   18 -
 .../main/resources/filters/summary.properties   |   32 -
 .../resources/filters/templates-min.properties  |   18 -
 .../main/resources/filters/templates.properties |   26 -
 .../main/resources/filters/users-min.properties |   18 -
 .../src/main/resources/filters/users.properties |   26 -
 .../webapp/WEB-INF/pages/bulletin-board.jsp     |   49 -
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |  117 -
 .../src/main/webapp/WEB-INF/pages/cluster.jsp   |   61 -
 .../src/main/webapp/WEB-INF/pages/counters.jsp  |   60 -
 .../src/main/webapp/WEB-INF/pages/history.jsp   |   64 -
 .../main/webapp/WEB-INF/pages/message-page.jsp  |   34 -
 .../main/webapp/WEB-INF/pages/provenance.jsp    |   67 -
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   79 -
 .../src/main/webapp/WEB-INF/pages/templates.jsp |   62 -
 .../src/main/webapp/WEB-INF/pages/users.jsp     |   70 -
 .../webapp/WEB-INF/partials/banners-main.jsp    |   19 -
 .../webapp/WEB-INF/partials/banners-utility.jsp |   19 -
 .../bulletin-board/bulletin-board-content.jsp   |   40 -
 .../WEB-INF/partials/canvas/about-dialog.jsp    |   29 -
 .../WEB-INF/partials/canvas/canvas-header.jsp   |   56 -
 .../canvas/connection-configuration.jsp         |  180 -
 .../partials/canvas/connections-dialog.jsp      |   35 -
 .../partials/canvas/fill-color-dialog.jsp       |   42 -
 .../WEB-INF/partials/canvas/flow-status.jsp     |   55 -
 .../canvas/instantiate-template-dialog.jsp      |   22 -
 .../partials/canvas/label-configuration.jsp     |   38 -
 .../WEB-INF/partials/canvas/navigation.jsp      |   57 -
 .../WEB-INF/partials/canvas/new-port-dialog.jsp |   27 -
 .../canvas/new-process-group-dialog.jsp         |   27 -
 .../partials/canvas/new-processor-dialog.jsp    |   50 -
 .../canvas/new-processor-property-dialog.jsp    |   34 -
 .../canvas/new-remote-process-group-dialog.jsp  |   27 -
 .../partials/canvas/new-template-dialog.jsp     |   33 -
 .../partials/canvas/port-configuration.jsp      |   54 -
 .../WEB-INF/partials/canvas/port-details.jsp    |   39 -
 .../canvas/process-group-configuration.jsp      |   34 -
 .../partials/canvas/process-group-details.jsp   |   33 -
 .../partials/canvas/processor-configuration.jsp |  213 --
 .../WEB-INF/partials/canvas/registration.jsp    |   44 -
 .../canvas/remote-port-configuration.jsp        |   42 -
 .../remote-process-group-configuration.jsp      |   50 -
 .../canvas/remote-process-group-details.jsp     |   48 -
 .../canvas/remote-process-group-ports.jsp       |   47 -
 .../canvas/secure-port-configuration.jsp        |   82 -
 .../partials/canvas/secure-port-details.jsp     |   67 -
 .../partials/canvas/settings-content.jsp        |   60 -
 .../webapp/WEB-INF/partials/canvas/shell.jsp    |   27 -
 .../partials/cluster/cluster-content.jsp        |   39 -
 .../partials/cluster/node-details-dialog.jsp    |   42 -
 .../WEB-INF/partials/connection-details.jsp     |  123 -
 .../partials/counters/counters-content.jsp      |   39 -
 .../partials/history/action-details-dialog.jsp  |   27 -
 .../partials/history/history-content.jsp        |   36 -
 .../partials/history/history-filter-dialog.jsp  |   73 -
 .../partials/history/history-purge-dialog.jsp   |   42 -
 .../webapp/WEB-INF/partials/message-pane.jsp    |   23 -
 .../main/webapp/WEB-INF/partials/ok-dialog.jsp  |   22 -
 .../WEB-INF/partials/processor-details.jsp      |  158 -
 .../provenance/lineage-query-dialog.jsp         |   26 -
 .../partials/provenance/provenance-content.jsp  |   73 -
 .../provenance-event-details-dialog.jsp         |  196 -
 .../provenance/provenance-query-dialog.jsp      |   26 -
 .../provenance/provenance-search-dialog.jsp     |  101 -
 .../WEB-INF/partials/status-history-dialog.jsp  |   37 -
 .../cluster-connection-summary-dialog.jsp       |   36 -
 .../cluster-input-port-summary-dialog.jsp       |   36 -
 .../cluster-output-port-summary-dialog.jsp      |   36 -
 .../cluster-processor-summary-dialog.jsp        |   36 -
 ...ster-remote-process-group-summary-dialog.jsp |   36 -
 .../partials/summary/summary-content.jsp        |   64 -
 .../summary/system-diagnostics-dialog.jsp       |  131 -
 .../summary/view-single-node-dialog.jsp         |   24 -
 .../partials/templates/templates-content.jsp    |   59 -
 .../partials/users/group-revoke-dialog.jsp      |   22 -
 .../partials/users/group-roles-dialog.jsp       |   52 -
 .../partials/users/user-delete-dialog.jsp       |   23 -
 .../partials/users/user-details-dialog.jsp      |   56 -
 .../partials/users/user-group-dialog.jsp        |   27 -
 .../partials/users/user-revoke-dialog.jsp       |   23 -
 .../partials/users/user-roles-dialog.jsp        |   60 -
 .../WEB-INF/partials/users/users-content.jsp    |   46 -
 .../webapp/WEB-INF/partials/yes-no-dialog.jsp   |   22 -
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |  120 -
 .../nifi-web-ui/src/main/webapp/css/about.css   |   54 -
 .../nifi-web-ui/src/main/webapp/css/banner.css  |   80 -
 .../src/main/webapp/css/bulletin-board.css      |  238 --
 .../nifi-web-ui/src/main/webapp/css/canvas.css  |   41 -
 .../nifi-web-ui/src/main/webapp/css/cluster.css |  152 -
 .../webapp/css/connection-configuration.css     |  168 -
 .../src/main/webapp/css/connection-details.css  |   77 -
 .../src/main/webapp/css/counters.css            |  134 -
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |  351 --
 .../src/main/webapp/css/flow-status.css         |  110 -
 .../nifi-web-ui/src/main/webapp/css/graph.css   |  262 --
 .../nifi-web-ui/src/main/webapp/css/header.css  |  683 ----
 .../nifi-web-ui/src/main/webapp/css/history.css |  216 --
 .../src/main/webapp/css/label-configuration.css |   55 -
 .../nifi-web-ui/src/main/webapp/css/main.css    |  298 --
 .../src/main/webapp/css/message-page.css        |   30 -
 .../src/main/webapp/css/message-pane.css        |   44 -
 .../src/main/webapp/css/navigation.css          |  292 --
 .../main/webapp/css/new-processor-dialog.css    |  168 -
 .../src/main/webapp/css/port-configuration.css  |  192 -
 .../src/main/webapp/css/port-details.css        |   55 -
 .../webapp/css/process-group-configuration.css  |   40 -
 .../main/webapp/css/process-group-details.css   |   27 -
 .../main/webapp/css/processor-configuration.css |  388 --
 .../src/main/webapp/css/processor-details.css   |  119 -
 .../src/main/webapp/css/provenance.css          |  654 ----
 .../src/main/webapp/css/registration.css        |   53 -
 .../css/remote-process-group-configuration.css  |  233 --
 .../nifi-web-ui/src/main/webapp/css/reset.css   |   43 -
 .../src/main/webapp/css/settings.css            |   66 -
 .../nifi-web-ui/src/main/webapp/css/shell.css   |   98 -
 .../src/main/webapp/css/status-history.css      |  199 -
 .../nifi-web-ui/src/main/webapp/css/summary.css |  768 ----
 .../src/main/webapp/css/templates.css           |  229 --
 .../nifi-web-ui/src/main/webapp/css/users.css   |  254 --
 .../src/main/webapp/images/addConnect.png       |  Bin 1996 -> 0 bytes
 .../src/main/webapp/images/bg-error.png         |  Bin 63766 -> 0 bytes
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 189 -> 0 bytes
 .../src/main/webapp/images/bgBannerHead.png     |  Bin 422 -> 0 bytes
 .../src/main/webapp/images/bgBreadcrumbs.png    |  Bin 2130 -> 0 bytes
 .../src/main/webapp/images/bgButton.png         |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgButtonOver.png     |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgButtonSelected.png |  Bin 166 -> 0 bytes
 .../main/webapp/images/bgButtonSelectedOver.png |  Bin 168 -> 0 bytes
 .../src/main/webapp/images/bgContextMenu.png    |  Bin 746 -> 0 bytes
 .../src/main/webapp/images/bgControlsInset.png  |  Bin 618 -> 0 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 1697 -> 0 bytes
 .../src/main/webapp/images/bgInputText.png      |  Bin 139 -> 0 bytes
 .../src/main/webapp/images/bgNifiLogo.png       |  Bin 4232 -> 0 bytes
 .../src/main/webapp/images/bgNifiToolbar.png    |  Bin 2123 -> 0 bytes
 .../main/webapp/images/bgNifiToolbarEdge.png    |  Bin 541 -> 0 bytes
 .../src/main/webapp/images/bgOutline.png        |  Bin 585 -> 0 bytes
 .../src/main/webapp/images/bgPanCenter.png      |  Bin 362 -> 0 bytes
 .../src/main/webapp/images/bgPanZoom.png        |  Bin 832 -> 0 bytes
 .../webapp/images/bgProcessGroupDetailsArea.png |  Bin 6596 -> 0 bytes
 .../main/webapp/images/bgProcessorStatArea.png  |  Bin 4062 -> 0 bytes
 .../images/bgRemoteProcessGroupDetailsArea.png  |  Bin 4433 -> 0 bytes
 .../src/main/webapp/images/bgSearchInput.png    |  Bin 221 -> 0 bytes
 .../src/main/webapp/images/bgShellClose.png     |  Bin 169 -> 0 bytes
 .../src/main/webapp/images/bgStatusPanel.png    |  Bin 229 -> 0 bytes
 .../src/main/webapp/images/bgTabContainer.png   |  Bin 234 -> 0 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 232 -> 0 bytes
 .../main/webapp/images/bgToolbarBtnBorder.png   |  Bin 227 -> 0 bytes
 .../src/main/webapp/images/blueBtnBg-over.jpg   |  Bin 356 -> 0 bytes
 .../src/main/webapp/images/blueBtnBg.jpg        |  Bin 356 -> 0 bytes
 .../src/main/webapp/images/buttonColor.png      |  Bin 1679 -> 0 bytes
 .../src/main/webapp/images/buttonCopy.png       |  Bin 1166 -> 0 bytes
 .../src/main/webapp/images/buttonDelete.png     |  Bin 1837 -> 0 bytes
 .../src/main/webapp/images/buttonDisable.png    |  Bin 1737 -> 0 bytes
 .../src/main/webapp/images/buttonEnable.png     |  Bin 1375 -> 0 bytes
 .../src/main/webapp/images/buttonGroup.png      |  Bin 880 -> 0 bytes
 .../main/webapp/images/buttonNewProperty.png    |  Bin 590 -> 0 bytes
 .../src/main/webapp/images/buttonOutline.png    |  Bin 495 -> 0 bytes
 .../src/main/webapp/images/buttonPanDown.png    |  Bin 554 -> 0 bytes
 .../src/main/webapp/images/buttonPanLeft.png    |  Bin 588 -> 0 bytes
 .../src/main/webapp/images/buttonPanRight.png   |  Bin 530 -> 0 bytes
 .../src/main/webapp/images/buttonPanUp.png      |  Bin 551 -> 0 bytes
 .../src/main/webapp/images/buttonPaste.png      |  Bin 1372 -> 0 bytes
 .../src/main/webapp/images/buttonRefresh.png    |  Bin 915 -> 0 bytes
 .../src/main/webapp/images/buttonRun.png        |  Bin 1163 -> 0 bytes
 .../src/main/webapp/images/buttonStop.png       |  Bin 869 -> 0 bytes
 .../src/main/webapp/images/buttonTemplate.png   |  Bin 1290 -> 0 bytes
 .../src/main/webapp/images/buttonZoom100.png    |  Bin 449 -> 0 bytes
 .../src/main/webapp/images/buttonZoomFit.png    |  Bin 499 -> 0 bytes
 .../src/main/webapp/images/buttonZoomIn.png     |  Bin 435 -> 0 bytes
 .../src/main/webapp/images/buttonZoomOut.png    |  Bin 339 -> 0 bytes
 .../src/main/webapp/images/cluster.png          |  Bin 326 -> 0 bytes
 .../src/main/webapp/images/grayBtnBg-over.jpg   |  Bin 319 -> 0 bytes
 .../src/main/webapp/images/grayBtnBg.jpg        |  Bin 317 -> 0 bytes
 .../nifi-web-ui/src/main/webapp/images/grid.gif |  Bin 135 -> 0 bytes
 .../src/main/webapp/images/headerTabBg.gif      |  Bin 131 -> 0 bytes
 .../src/main/webapp/images/iconAdminUser.png    |  Bin 1960 -> 0 bytes
 .../src/main/webapp/images/iconAlert.png        |  Bin 1396 -> 0 bytes
 .../src/main/webapp/images/iconAlertDialog.png  |  Bin 1241 -> 0 bytes
 .../src/main/webapp/images/iconAutoRefresh.png  |  Bin 3102 -> 0 bytes
 .../src/main/webapp/images/iconBulletin.png     |  Bin 1066 -> 0 bytes
 .../main/webapp/images/iconBulletinBoard.png    |  Bin 2523 -> 0 bytes
 .../src/main/webapp/images/iconCenterView.png   |  Bin 338 -> 0 bytes
 .../src/main/webapp/images/iconChart.png        |  Bin 510 -> 0 bytes
 .../src/main/webapp/images/iconClose.png        |  Bin 231 -> 0 bytes
 .../src/main/webapp/images/iconCluster.png      |  Bin 2112 -> 0 bytes
 .../src/main/webapp/images/iconClusterSmall.png |  Bin 757 -> 0 bytes
 .../main/webapp/images/iconCollapseChildren.png |  Bin 571 -> 0 bytes
 .../main/webapp/images/iconCollapseParents.png  |  Bin 645 -> 0 bytes
 .../src/main/webapp/images/iconColor.png        |  Bin 738 -> 0 bytes
 .../src/main/webapp/images/iconCommit.png       |  Bin 445 -> 0 bytes
 .../src/main/webapp/images/iconConfigure.png    |  Bin 696 -> 0 bytes
 .../src/main/webapp/images/iconConnect.png      |  Bin 589 -> 0 bytes
 .../src/main/webapp/images/iconConnection.png   |  Bin 1517 -> 0 bytes
 .../src/main/webapp/images/iconCopy.png         |  Bin 514 -> 0 bytes
 .../src/main/webapp/images/iconCounter.png      |  Bin 1369 -> 0 bytes
 .../src/main/webapp/images/iconDelete.png       |  Bin 670 -> 0 bytes
 .../src/main/webapp/images/iconDetails.png      |  Bin 549 -> 0 bytes
 .../src/main/webapp/images/iconDisable.png      |  Bin 764 -> 0 bytes
 .../src/main/webapp/images/iconDisconnect.png   |  Bin 569 -> 0 bytes
 .../src/main/webapp/images/iconEdit.png         |  Bin 493 -> 0 bytes
 .../src/main/webapp/images/iconEditButton.png   |  Bin 915 -> 0 bytes
 .../main/webapp/images/iconExpandChildren.png   |  Bin 521 -> 0 bytes
 .../main/webapp/images/iconExpandParents.png    |  Bin 585 -> 0 bytes
 .../src/main/webapp/images/iconExport.png       |  Bin 453 -> 0 bytes
 .../main/webapp/images/iconExportLineage.png    |  Bin 405 -> 0 bytes
 .../src/main/webapp/images/iconFlowSettings.png |  Bin 2624 -> 0 bytes
 .../src/main/webapp/images/iconFunnel.png       |  Bin 1223 -> 0 bytes
 .../src/main/webapp/images/iconGoTo.png         |  Bin 448 -> 0 bytes
 .../src/main/webapp/images/iconHistory.png      |  Bin 2347 -> 0 bytes
 .../src/main/webapp/images/iconInfo.png         |  Bin 550 -> 0 bytes
 .../src/main/webapp/images/iconInputPort.png    |  Bin 1842 -> 0 bytes
 .../main/webapp/images/iconInputPortSmall.png   |  Bin 532 -> 0 bytes
 .../webapp/images/iconIsolatedProcessor.png     |  Bin 1781 -> 0 bytes
 .../src/main/webapp/images/iconLineage.png      |  Bin 432 -> 0 bytes
 .../src/main/webapp/images/iconLoading.gif      |  Bin 2438 -> 0 bytes
 .../src/main/webapp/images/iconNotSecure.png    |  Bin 137 -> 0 bytes
 .../src/main/webapp/images/iconOutputPort.png   |  Bin 1658 -> 0 bytes
 .../main/webapp/images/iconOutputPortSmall.png  |  Bin 459 -> 0 bytes
 .../src/main/webapp/images/iconPaste.png        |  Bin 601 -> 0 bytes
 .../src/main/webapp/images/iconPopOut.png       |  Bin 217 -> 0 bytes
 .../webapp/images/iconPortNotTransmitting.png   |  Bin 231 -> 0 bytes
 .../src/main/webapp/images/iconPortRemoved.png  |  Bin 456 -> 0 bytes
 .../main/webapp/images/iconPortTransmitting.png |  Bin 235 -> 0 bytes
 .../src/main/webapp/images/iconPrimary.png      |  Bin 647 -> 0 bytes
 .../src/main/webapp/images/iconProcessor.png    |  Bin 1446 -> 0 bytes
 .../src/main/webapp/images/iconProvenance.png   |  Bin 1104 -> 0 bytes
 .../src/main/webapp/images/iconRefresh.png      |  Bin 492 -> 0 bytes
 .../src/main/webapp/images/iconRemotePorts.png  |  Bin 456 -> 0 bytes
 .../webapp/images/iconRemoteProcessGroup.png    |  Bin 674 -> 0 bytes
 .../src/main/webapp/images/iconReporting.png    |  Bin 803 -> 0 bytes
 .../src/main/webapp/images/iconResetCounter.png |  Bin 304 -> 0 bytes
 .../src/main/webapp/images/iconResize.png       |  Bin 142 -> 0 bytes
 .../src/main/webapp/images/iconRevoke.png       |  Bin 676 -> 0 bytes
 .../src/main/webapp/images/iconRun.png          |  Bin 538 -> 0 bytes
 .../src/main/webapp/images/iconSecure.png       |  Bin 133 -> 0 bytes
 .../src/main/webapp/images/iconSmallFunnel.png  |  Bin 455 -> 0 bytes
 .../main/webapp/images/iconSmallInputPort.png   |  Bin 216 -> 0 bytes
 .../main/webapp/images/iconSmallOutputPort.png  |  Bin 264 -> 0 bytes
 .../webapp/images/iconSmallProcessGroup.png     |  Bin 479 -> 0 bytes
 .../main/webapp/images/iconSmallProcessor.png   |  Bin 647 -> 0 bytes
 .../webapp/images/iconSmallRelationship.png     |  Bin 770 -> 0 bytes
 .../images/iconSmallRemoteProcessGroup.png      |  Bin 402 -> 0 bytes
 .../src/main/webapp/images/iconStop.png         |  Bin 402 -> 0 bytes
 .../src/main/webapp/images/iconTemplates.png    |  Bin 2659 -> 0 bytes
 .../src/main/webapp/images/iconToFront.png      |  Bin 475 -> 0 bytes
 .../webapp/images/iconTransmissionActive.png    |  Bin 1330 -> 0 bytes
 .../webapp/images/iconTransmissionInactive.png  |  Bin 1248 -> 0 bytes
 .../src/main/webapp/images/iconTwistArrow.png   |  Bin 256 -> 0 bytes
 .../src/main/webapp/images/iconUndo.png         |  Bin 642 -> 0 bytes
 .../src/main/webapp/images/iconUsage.png        |  Bin 470 -> 0 bytes
 .../src/main/webapp/images/inputCheckbox.png    |  Bin 389 -> 0 bytes
 .../src/main/webapp/images/loadAnimation.gif    |  Bin 10789 -> 0 bytes
 .../src/main/webapp/images/nifi16.ico           |  Bin 1150 -> 0 bytes
 .../src/main/webapp/images/nifi_about.png       |  Bin 16323 -> 0 bytes
 .../src/main/webapp/images/panelBg.jpg          |  Bin 312 -> 0 bytes
 .../src/main/webapp/images/portRemoved.png      |  Bin 1090 -> 0 bytes
 .../src/main/webapp/images/portRunning.png      |  Bin 337 -> 0 bytes
 .../src/main/webapp/images/portStopped.png      |  Bin 192 -> 0 bytes
 .../src/main/webapp/images/spacer.png           |  Bin 110 -> 0 bytes
 .../src/main/webapp/images/starburst.png        |  Bin 190 -> 0 bytes
 .../src/main/webapp/images/tabBg.jpg            |  Bin 320 -> 0 bytes
 .../src/main/webapp/images/toolbarBg.gif        |  Bin 45 -> 0 bytes
 .../main/webapp/images/toolboxIconFunnel.png    |  Bin 2409 -> 0 bytes
 .../main/webapp/images/toolboxIconInputPort.png |  Bin 2680 -> 0 bytes
 .../src/main/webapp/images/toolboxIconLabel.png |  Bin 2871 -> 0 bytes
 .../webapp/images/toolboxIconOutputPort.png     |  Bin 2209 -> 0 bytes
 .../webapp/images/toolboxIconProcessGroup.png   |  Bin 4176 -> 0 bytes
 .../main/webapp/images/toolboxIconProcessor.png |  Bin 4426 -> 0 bytes
 .../images/toolboxIconRemoteProcessGroup.png    |  Bin 5653 -> 0 bytes
 .../main/webapp/images/toolboxIconTemplate.png  |  Bin 2913 -> 0 bytes
 .../images/transmissionSwitchDisabled.png       |  Bin 1240 -> 0 bytes
 .../webapp/images/transmissionSwitchEnabled.png |  Bin 1600 -> 0 bytes
 .../src/main/webapp/images/ungroup.png          |  Bin 3409 -> 0 bytes
 .../src/main/webapp/js/codemirror/LICENSE       |   19 -
 .../js/codemirror/addon/hint/show-hint.css      |   38 -
 .../js/codemirror/lib/codemirror-compressed.js  |   16 -
 .../webapp/js/codemirror/lib/codemirror.css     |  318 --
 .../nifi-web-ui/src/main/webapp/js/d3/LICENSE   |   26 -
 .../nifi-web-ui/src/main/webapp/js/d3/d3.min.js |    5 -
 .../webapp/js/jquery/combo/jquery.combo.css     |   82 -
 .../main/webapp/js/jquery/combo/jquery.combo.js |  311 --
 .../main/webapp/js/jquery/combo/menuArrow.png   |  Bin 251 -> 0 bytes
 .../webapp/js/jquery/combo/menuArrowSmall.png   |  Bin 233 -> 0 bytes
 .../main/webapp/js/jquery/jquery-2.1.1.min.js   |    4 -
 .../src/main/webapp/js/jquery/jquery.center.js  |   35 -
 .../src/main/webapp/js/jquery/jquery.count.js   |   99 -
 .../src/main/webapp/js/jquery/jquery.each.js    |   31 -
 .../main/webapp/js/jquery/jquery.ellipsis.js    |  164 -
 .../js/jquery/jquery.event.drag-2.2.min.js      |    6 -
 .../main/webapp/js/jquery/jquery.form.min.js    |   11 -
 .../src/main/webapp/js/jquery/jquery.tab.js     |   69 -
 .../js/jquery/minicolors/jquery.minicolors.css  |  259 --
 .../jquery/minicolors/jquery.minicolors.min.js  |    9 -
 .../js/jquery/minicolors/jquery.minicolors.png  |  Bin 77459 -> 0 bytes
 .../webapp/js/jquery/modal/jquery.modal.css     |  110 -
 .../main/webapp/js/jquery/modal/jquery.modal.js |  266 --
 .../js/jquery/nfeditor/jquery.nfeditor.css      |   73 -
 .../js/jquery/nfeditor/jquery.nfeditor.js       |  315 --
 .../js/jquery/nfeditor/languages/nfel.css       |   51 -
 .../webapp/js/jquery/nfeditor/languages/nfel.js |  831 ----
 .../webapp/js/jquery/qtip2/jquery.qtip.min.css  |    3 -
 .../webapp/js/jquery/qtip2/jquery.qtip.min.js   |    5 -
 .../webapp/js/jquery/slickgrid/MIT-LICENSE.txt  |   20 -
 .../js/jquery/slickgrid/css/images/collapse.gif |  Bin 846 -> 0 bytes
 .../js/jquery/slickgrid/css/images/expand.gif   |  Bin 851 -> 0 bytes
 .../js/jquery/slickgrid/css/images/sort-asc.gif |  Bin 59 -> 0 bytes
 .../jquery/slickgrid/css/images/sort-desc.gif   |  Bin 59 -> 0 bytes
 .../slickgrid/css/slick-default-theme.css       |  135 -
 .../js/jquery/slickgrid/css/slick.grid.css      |  158 -
 .../slickgrid/plugins/slick.autotooltips.js     |   83 -
 .../plugins/slick.cellrangedecorator.js         |   66 -
 .../plugins/slick.cellrangeselector.js          |  113 -
 .../plugins/slick.cellselectionmodel.js         |  154 -
 .../plugins/slick.rowselectionmodel.js          |  187 -
 .../webapp/js/jquery/slickgrid/slick.core.js    |  467 ---
 .../js/jquery/slickgrid/slick.dataview.js       | 1126 ------
 .../webapp/js/jquery/slickgrid/slick.editors.js |  512 ---
 .../js/jquery/slickgrid/slick.formatters.js     |   59 -
 .../webapp/js/jquery/slickgrid/slick.grid.js    | 3422 -----------------
 .../webapp/js/jquery/tabbs/jquery.tabbs.css     |   48 -
 .../main/webapp/js/jquery/tabbs/jquery.tabbs.js |   96 -
 .../ui-smoothness/images/animated-overlay.gif   |  Bin 1738 -> 0 bytes
 .../images/ui-bg_flat_0_aaaaaa_40x100.png       |  Bin 212 -> 0 bytes
 .../images/ui-bg_flat_75_ffffff_40x100.png      |  Bin 208 -> 0 bytes
 .../images/ui-bg_glass_55_fbf9ee_1x400.png      |  Bin 335 -> 0 bytes
 .../images/ui-bg_glass_65_ffffff_1x400.png      |  Bin 207 -> 0 bytes
 .../images/ui-bg_glass_75_dadada_1x400.png      |  Bin 262 -> 0 bytes
 .../images/ui-bg_glass_75_e6e6e6_1x400.png      |  Bin 262 -> 0 bytes
 .../images/ui-bg_glass_95_fef1ec_1x400.png      |  Bin 332 -> 0 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    |  Bin 280 -> 0 bytes
 .../images/ui-icons_222222_256x240.png          |  Bin 6922 -> 0 bytes
 .../images/ui-icons_2e83ff_256x240.png          |  Bin 4549 -> 0 bytes
 .../images/ui-icons_454545_256x240.png          |  Bin 6992 -> 0 bytes
 .../images/ui-icons_888888_256x240.png          |  Bin 6999 -> 0 bytes
 .../images/ui-icons_cd0a0a_256x240.png          |  Bin 4549 -> 0 bytes
 .../ui-smoothness/jquery-ui-1.10.4.min.css      |    7 -
 .../ui-smoothness/jquery-ui-1.10.4.min.js       |    7 -
 .../web/nifi-web-ui/src/main/webapp/js/json2.js |  475 ---
 .../js/nf/bulletin-board/nf-bulletin-board.js   |  422 --
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 1150 ------
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  340 --
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  293 --
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  174 -
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    | 1211 ------
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 -------
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 1530 --------
 .../main/webapp/js/nf/canvas/nf-clipboard.js    |   96 -
 .../main/webapp/js/nf/canvas/nf-connectable.js  |  210 -
 .../js/nf/canvas/nf-connection-configuration.js | 1381 -------
 .../main/webapp/js/nf/canvas/nf-connection.js   | 1533 --------
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  452 ---
 .../js/nf/canvas/nf-custom-processor-ui.js      |   43 -
 .../main/webapp/js/nf/canvas/nf-draggable.js    |  319 --
 .../src/main/webapp/js/nf/canvas/nf-funnel.js   |  279 --
 .../src/main/webapp/js/nf/canvas/nf-go-to.js    |  730 ----
 .../webapp/js/nf/canvas/nf-graph-control.js     |  137 -
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |  194 -
 .../js/nf/canvas/nf-label-configuration.js      |  132 -
 .../src/main/webapp/js/nf/canvas/nf-label.js    |  559 ---
 .../js/nf/canvas/nf-port-configuration.js       |  170 -
 .../main/webapp/js/nf/canvas/nf-port-details.js |   63 -
 .../src/main/webapp/js/nf/canvas/nf-port.js     |  632 ---
 .../nf/canvas/nf-process-group-configuration.js |  107 -
 .../js/nf/canvas/nf-process-group-details.js    |   61 -
 .../webapp/js/nf/canvas/nf-process-group.js     | 1064 -----
 .../js/nf/canvas/nf-processor-configuration.js  |  734 ----
 .../nf-processor-property-combo-editor.js       |  175 -
 .../canvas/nf-processor-property-nfel-editor.js |  207 -
 .../js/nf/canvas/nf-processor-property-table.js |  567 ---
 .../canvas/nf-processor-property-text-editor.js |  212 -
 .../main/webapp/js/nf/canvas/nf-processor.js    |  831 ----
 .../main/webapp/js/nf/canvas/nf-registration.js |   68 -
 .../nf-remote-process-group-configuration.js    |  124 -
 .../canvas/nf-remote-process-group-details.js   |   67 -
 .../nf/canvas/nf-remote-process-group-ports.js  |  525 ---
 .../js/nf/canvas/nf-remote-process-group.js     | 1060 -----
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  179 -
 .../nf/canvas/nf-secure-port-configuration.js   |  384 --
 .../js/nf/canvas/nf-secure-port-details.js      |  118 -
 .../main/webapp/js/nf/canvas/nf-selectable.js   |   58 -
 .../src/main/webapp/js/nf/canvas/nf-settings.js |  146 -
 .../src/main/webapp/js/nf/canvas/nf-snippet.js  |  212 -
 .../src/main/webapp/js/nf/canvas/nf-storage.js  |  136 -
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |   86 -
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  620 ---
 .../src/main/webapp/js/nf/cluster/nf-cluster.js |  154 -
 .../webapp/js/nf/counters/nf-counters-table.js  |  294 --
 .../main/webapp/js/nf/counters/nf-counters.js   |  153 -
 .../webapp/js/nf/history/nf-history-model.js    |  218 --
 .../webapp/js/nf/history/nf-history-table.js    |  429 ---
 .../src/main/webapp/js/nf/history/nf-history.js |  154 -
 .../src/main/webapp/js/nf/nf-client.js          |   56 -
 .../src/main/webapp/js/nf/nf-common.js          |  899 -----
 .../main/webapp/js/nf/nf-connection-details.js  |  467 ---
 .../src/main/webapp/js/nf/nf-dialog.js          |  128 -
 .../src/main/webapp/js/nf/nf-namespace.js       |   20 -
 .../main/webapp/js/nf/nf-processor-details.js   |  586 ---
 .../src/main/webapp/js/nf/nf-shell.js           |  167 -
 .../src/main/webapp/js/nf/nf-status-history.js  | 1320 -------
 .../js/nf/provenance/nf-provenance-lineage.js   | 1413 -------
 .../js/nf/provenance/nf-provenance-table.js     | 1371 -------
 .../webapp/js/nf/provenance/nf-provenance.js    |  204 -
 .../webapp/js/nf/summary/nf-cluster-search.js   |  183 -
 .../webapp/js/nf/summary/nf-summary-table.js    | 2436 ------------
 .../src/main/webapp/js/nf/summary/nf-summary.js |  174 -
 .../js/nf/templates/nf-templates-table.js       |  332 --
 .../main/webapp/js/nf/templates/nf-templates.js |  235 --
 .../main/webapp/js/nf/users/nf-users-table.js   | 1090 ------
 .../src/main/webapp/js/nf/users/nf-users.js     |  149 -
 .../framework-bundle/framework/web/pom.xml      |   74 -
 .../web/web-optimistic-locking/.gitignore       |    1 -
 .../web/web-optimistic-locking/pom.xml          |   32 -
 .../apache/nifi/web/ConfigurationSnapshot.java  |   66 -
 .../nifi/web/OptimisticLockingManager.java      |   95 -
 .../web/StandardOptimisticLockingManager.java   |   77 -
 .../framework/web/web-security/.gitignore       |    1 -
 .../framework/web/web-security/pom.xml          |   81 -
 .../org/apache/nifi/web/security/DnUtils.java   |   87 -
 .../web/security/UntrustedProxyException.java   |   34 -
 .../anonymous/NiFiAnonymousUserFilter.java      |  101 -
 .../NiFiAuthenticationEntryPoint.java           |   68 -
 .../authorization/NiFiAuthorizationService.java |  173 -
 .../authorization/NodeAuthorizedUserFilter.java |  128 -
 .../nifi/web/security/user/NiFiUserDetails.java |  110 -
 .../nifi/web/security/user/NiFiUserUtils.java   |   82 -
 .../x509/SubjectDnX509PrincipalExtractor.java   |   38 -
 .../security/x509/X509AuthenticationFilter.java |  349 --
 .../security/x509/X509CertificateExtractor.java |   52 -
 .../x509/ocsp/CertificateStatusException.java   |   42 -
 .../x509/ocsp/OcspCertificateValidator.java     |  444 ---
 .../web/security/x509/ocsp/OcspRequest.java     |   69 -
 .../nifi/web/security/x509/ocsp/OcspStatus.java |   82 -
 .../resources/nifi-web-security-context.xml     |   83 -
 .../NiFiAuthorizationServiceTest.java           |  250 --
 nar-bundles/framework-bundle/nar/.gitignore     |    1 -
 nar-bundles/framework-bundle/nar/pom.xml        |   66 -
 nar-bundles/framework-bundle/pom.xml            |  114 -
 .../hadoop-bundle/hdfs-processors/pom.xml       |   65 -
 .../hadoop/AbstractHadoopProcessor.java         |  221 --
 .../hadoop/CreateHadoopSequenceFile.java        |  177 -
 ...lowFileStreamUnpackerSequenceFileWriter.java |  170 -
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  563 ---
 .../processors/hadoop/GetHDFSSequenceFile.java  |  146 -
 .../nifi/processors/hadoop/KeyValueReader.java  |  132 -
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  403 --
 .../hadoop/SequenceFileWriterImpl.java          |  119 -
 .../hadoop/TarUnpackerSequenceFileWriter.java   |   54 -
 .../nifi/processors/hadoop/ValueReader.java     |  116 -
 .../hadoop/ZipUnpackerSequenceFileWriter.java   |   57 -
 .../hadoop/util/ByteFilteringOutputStream.java  |  163 -
 .../hadoop/util/InputStreamWritable.java        |   61 -
 .../hadoop/util/OutputStreamWritable.java       |   85 -
 .../hadoop/util/SequenceFileReader.java         |   28 -
 .../hadoop/util/SequenceFileWriter.java         |   40 -
 .../org.apache.nifi.processor.Processor         |   18 -
 .../index.html                                  |   88 -
 .../index.html                                  |  162 -
 .../index.html                                  |  150 -
 .../index.html                                  |  159 -
 .../processors/hadoop/AbstractHadoopTest.java   |   90 -
 .../nifi/processors/hadoop/GetHDFSTest.java     |  123 -
 .../nifi/processors/hadoop/PutHDFSTest.java     |  158 -
 .../hadoop/SimpleHadoopProcessor.java           |   30 -
 .../hadoop/TestCreateHadoopSequenceFile.java    |  176 -
 .../src/test/resources/core-site-broken.xml     |   25 -
 .../src/test/resources/core-site.xml            |   25 -
 .../resources/testdata/13545312236534130.tar    |  Bin 1505280 -> 0 bytes
 .../resources/testdata/13545423550275052.zip    |  Bin 1500841 -> 0 bytes
 .../resources/testdata/13545479542069498.pkg    |  Bin 1500429 -> 0 bytes
 .../src/test/resources/testdata/randombytes-1   |  Bin 500000 -> 0 bytes
 .../src/test/resources/testdata/randombytes-2   |  Bin 500000 -> 0 bytes
 .../src/test/resources/testdata/randombytes-3   |  Bin 500000 -> 0 bytes
 nar-bundles/hadoop-bundle/nar/pom.xml           |   41 -
 nar-bundles/hadoop-bundle/pom.xml               |   46 -
 nar-bundles/hadoop-libraries-bundle/nar/pom.xml |   45 -
 nar-bundles/hadoop-libraries-bundle/pom.xml     |   36 -
 nar-bundles/jetty-bundle/pom.xml                |   82 -
 nar-bundles/kafka-bundle/kafka-nar/pom.xml      |   37 -
 .../kafka-bundle/kafka-processors/pom.xml       |   76 -
 .../apache/nifi/processors/kafka/GetKafka.java  |  330 --
 .../apache/nifi/processors/kafka/PutKafka.java  |  419 --
 .../org.apache.nifi.processor.Processor         |   16 -
 .../index.html                                  |  173 -
 .../index.html                                  |  189 -
 .../nifi/processors/kafka/TestGetKafka.java     |  162 -
 .../nifi/processors/kafka/TestPutKafka.java     |  236 --
 nar-bundles/kafka-bundle/pom.xml                |   35 -
 .../monitor-threshold-bundle/nar/pom.xml        |   39 -
 nar-bundles/monitor-threshold-bundle/pom.xml    |   51 -
 .../monitor-threshold-bundle/processor/pom.xml  |   45 -
 .../processors/monitor/MonitorThreshold.java    |  904 -----
 .../processors/monitor/ThresholdsParser.java    |  134 -
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  485 ---
 .../monitor/TestMonitorThreshold.java           | 1560 --------
 .../processor/src/test/resources/testFile       |   23 -
 .../resources/threshold_settings_allow_0.xml    |   21 -
 .../resources/threshold_settings_allow_1.xml    |   21 -
 .../threshold_settings_allow_10_KB.xml          |   21 -
 .../threshold_settings_allow_10_KB_and_2_KB.xml |   26 -
 .../threshold_settings_allow_1669_bytes.xml     |   21 -
 .../resources/threshold_settings_allow_2.xml    |   21 -
 .../threshold_settings_allow_2_default.xml      |   21 -
 ...eshold_settings_allow_3336_bytes_default.xml |   21 -
 ...hold_settings_with_empty_attribute_value.xml |   21 -
 ...ettings_with_spaces_for_attribute_values.xml |   22 -
 .../ui/nb-configuration.xml                     |   18 -
 nar-bundles/monitor-threshold-bundle/ui/pom.xml |  134 -
 .../thresholds/ui/AttributeComparator.java      |   36 -
 .../thresholds/ui/AttributeResource.java        |  242 --
 .../monitor/thresholds/ui/RuleComparator.java   |   46 -
 .../thresholds/ui/ThresholdResource.java        |  227 --
 .../ui/ThresholdSettingsResource.java           |  191 -
 .../thresholds/ui/ThresholdsConfigFile.java     |  296 --
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 -
 .../webapp/WEB-INF/jsp/attribute-add-dialog.jsp |   43 -
 .../WEB-INF/jsp/attribute-confirm-dialog.jsp    |   21 -
 .../WEB-INF/jsp/attribute-edit-dialog.jsp       |   27 -
 .../WEB-INF/jsp/attribute-filter-dialog.jsp     |   26 -
 .../main/webapp/WEB-INF/jsp/error-dialog.jsp    |   21 -
 .../ui/src/main/webapp/WEB-INF/jsp/index.jsp    |  101 -
 .../webapp/WEB-INF/jsp/threshold-add-dialog.jsp |   45 -
 .../WEB-INF/jsp/threshold-confirm-dialog.jsp    |   21 -
 .../webapp/WEB-INF/jsp/threshold-dialog.jsp     |   48 -
 .../WEB-INF/jsp/threshold-edit-dialog.jsp       |   45 -
 .../WEB-INF/jsp/threshold-filter-dialog.jsp     |   40 -
 .../ui/src/main/webapp/WEB-INF/web.xml          |   48 -
 .../ui/src/main/webapp/css/threshold_styles.css |  422 --
 .../src/main/webapp/images/addWorksheetRow.png  |  Bin 323 -> 0 bytes
 .../ui/src/main/webapp/images/clear.png         |  Bin 912 -> 0 bytes
 .../ui/src/main/webapp/images/filter.gif        |  Bin 623 -> 0 bytes
 .../ui/src/main/webapp/images/firstPage.gif     |  Bin 577 -> 0 bytes
 .../main/webapp/images/firstPageDisabled.gif    |  Bin 365 -> 0 bytes
 .../ui/src/main/webapp/images/lastPage.gif      |  Bin 579 -> 0 bytes
 .../src/main/webapp/images/lastPageDisabled.gif |  Bin 368 -> 0 bytes
 .../ui/src/main/webapp/images/nextPage.gif      |  Bin 570 -> 0 bytes
 .../src/main/webapp/images/nextPageDisabled.gif |  Bin 362 -> 0 bytes
 .../ui/src/main/webapp/images/prevPage.gif      |  Bin 567 -> 0 bytes
 .../src/main/webapp/images/prevPageDisabled.gif |  Bin 361 -> 0 bytes
 .../main/webapp/images/removeWorksheetRow.png   |  Bin 655 -> 0 bytes
 .../ui/src/main/webapp/images/separator.gif     |  Bin 63 -> 0 bytes
 .../webapp/js/jquery/jqgrid/css/ui.jqgrid.css   |  171 -
 .../js/jquery/jqgrid/js/i18n/grid.locale-en.js  |  170 -
 .../js/jquery/jqgrid/js/jquery.jqGrid.min.js    |  545 ---
 .../ui/src/main/webapp/js/nf-common.js          |  888 -----
 .../ui/src/main/xsd/threshold_settings.xsd      |   54 -
 .../nar/pom.xml                                 |   36 -
 .../persistent-provenance-repository/pom.xml    |   58 -
 .../nifi/provenance/IndexConfiguration.java     |  398 --
 .../PersistentProvenanceRepository.java         | 1892 ---------
 .../provenance/RepositoryConfiguration.java     |  299 --
 .../nifi/provenance/StandardRecordReader.java   |  305 --
 .../nifi/provenance/StandardRecordWriter.java   |  237 --
 .../provenance/expiration/ExpirationAction.java |   35 -
 .../expiration/FileRemovalAction.java           |   49 -
 .../provenance/lucene/DeleteIndexAction.java    |  139 -
 .../nifi/provenance/lucene/DocsReader.java      |  133 -
 .../nifi/provenance/lucene/FieldNames.java      |   23 -
 .../nifi/provenance/lucene/IndexSearch.java     |   84 -
 .../nifi/provenance/lucene/IndexingAction.java  |  219 --
 .../nifi/provenance/lucene/LineageQuery.java    |  108 -
 .../nifi/provenance/lucene/LuceneUtil.java      |  141 -
 .../provenance/rollover/CompressionAction.java  |   59 -
 .../provenance/rollover/RolloverAction.java     |   35 -
 .../provenance/serialization/RecordReader.java  |   31 -
 .../provenance/serialization/RecordReaders.java |   70 -
 .../provenance/serialization/RecordWriter.java  |   85 -
 .../provenance/serialization/RecordWriters.java |   30 -
 ...he.nifi.provenance.ProvenanceEventRepository |   15 -
 .../TestPersistentProvenanceRepository.java     | 1118 ------
 .../pom.xml                                     |   42 -
 nar-bundles/pom.xml                             |  134 -
 .../standard-bundle/jms-processors/pom.xml      |   55 -
 .../apache/nifi/processors/jms/GetJMSQueue.java |   75 -
 .../apache/nifi/processors/jms/GetJMSTopic.java |  359 --
 .../apache/nifi/processors/jms/JmsConsumer.java |  208 -
 .../org/apache/nifi/processors/jms/PutJMS.java  |  374 --
 .../nifi/processors/jms/util/JmsFactory.java    |  435 ---
 .../nifi/processors/jms/util/JmsProperties.java |  176 -
 .../jms/util/WrappedMessageConsumer.java        |   77 -
 .../jms/util/WrappedMessageProducer.java        |   77 -
 .../org.apache.nifi.processor.Processor         |   17 -
 .../index.html                                  |  118 -
 .../index.html                                  |  122 -
 .../index.html                                  |  152 -
 .../nifi/processors/jms/GetJMSQueueTest.java    |  141 -
 nar-bundles/standard-bundle/nar/pom.xml         |   57 -
 nar-bundles/standard-bundle/pom.xml             |   71 -
 .../standard-ganglia-reporter/pom.xml           |   52 -
 .../ganglia/StandardGangliaReporter.java        |  262 --
 .../org.apache.nifi.reporting.ReportingTask     |   15 -
 .../index.html                                  |   65 -
 .../standard-prioritizers/pom.xml               |   40 -
 .../prioritizer/FirstInFirstOutPrioritizer.java |   37 -
 .../NewestFlowFileFirstPrioritizer.java         |   37 -
 .../OldestFlowFileFirstPrioritizer.java         |   37 -
 .../PriorityAttributePrioritizer.java           |   87 -
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   18 -
 .../prioritizer/NewestFirstPrioritizerTest.java |   66 -
 .../prioritizer/OldestFirstPrioritizerTest.java |   66 -
 .../PriorityAttributePrioritizerTest.java       |  117 -
 .../standard-bundle/standard-processors/pom.xml |  154 -
 .../standard/Base64EncodeContent.java           |  142 -
 .../processors/standard/CompressContent.java    |  307 --
 .../nifi/processors/standard/ControlRate.java   |  381 --
 .../standard/ConvertCharacterSet.java           |  175 -
 .../processors/standard/DetectDuplicate.java    |  249 --
 .../processors/standard/DistributeLoad.java     |  498 ---
 .../processors/standard/EncryptContent.java     |  263 --
 .../standard/EvaluateRegularExpression.java     |  294 --
 .../nifi/processors/standard/EvaluateXPath.java |  429 ---
 .../processors/standard/EvaluateXQuery.java     |  463 ---
 .../standard/ExecuteStreamCommand.java          |  358 --
 .../processors/standard/GenerateFlowFile.java   |  164 -
 .../apache/nifi/processors/standard/GetFTP.java |   72 -
 .../nifi/processors/standard/GetFile.java       |  456 ---
 .../processors/standard/GetFileTransfer.java    |  301 --
 .../nifi/processors/standard/GetHTTP.java       |  480 ---
 .../nifi/processors/standard/GetSFTP.java       |   92 -
 .../nifi/processors/standard/HashAttribute.java |  255 --
 .../nifi/processors/standard/HashContent.java   |  151 -
 .../processors/standard/IdentifyMimeType.java   |  474 ---
 .../nifi/processors/standard/InvokeHTTP.java    |  758 ----
 .../nifi/processors/standard/ListenHTTP.java    |  323 --
 .../nifi/processors/standard/ListenUDP.java     |  627 ---
 .../nifi/processors/standard/LogAttribute.java  |  262 --
 .../nifi/processors/standard/MergeContent.java  | 1015 -----
 .../nifi/processors/standard/ModifyBytes.java   |  134 -
 .../processors/standard/MonitorActivity.java    |  206 -
 .../nifi/processors/standard/PostHTTP.java      |  904 -----
 .../nifi/processors/standard/PutEmail.java      |  297 --
 .../apache/nifi/processors/standard/PutFTP.java |  141 -
 .../nifi/processors/standard/PutFile.java       |  367 --
 .../processors/standard/PutFileTransfer.java    |  293 --
 .../nifi/processors/standard/PutSFTP.java       |   85 -
 .../nifi/processors/standard/ReplaceText.java   |  289 --
 .../standard/ReplaceTextWithMapping.java        |  383 --
 .../processors/standard/RouteOnAttribute.java   |  261 --
 .../processors/standard/RouteOnContent.java     |  232 --
 .../nifi/processors/standard/ScanAttribute.java |  229 --
 .../nifi/processors/standard/ScanContent.java   |  292 --
 .../processors/standard/SegmentContent.java     |  163 -
 .../nifi/processors/standard/SplitContent.java  |  260 --
 .../nifi/processors/standard/SplitText.java     |  370 --
 .../nifi/processors/standard/SplitXml.java      |  300 --
 .../nifi/processors/standard/TransformXml.java  |  194 -
 .../nifi/processors/standard/UnpackContent.java |  427 ---
 .../nifi/processors/standard/ValidateXml.java   |  147 -
 .../servlets/ContentAcknowledgmentServlet.java  |  136 -
 .../standard/servlets/ListenHTTPServlet.java    |  325 --
 .../nifi/processors/standard/util/Bin.java      |  168 -
 .../processors/standard/util/BinManager.java    |  241 --
 .../standard/util/DocumentReaderCallback.java   |   74 -
 .../processors/standard/util/FTPTransfer.java   |  541 ---
 .../nifi/processors/standard/util/FTPUtils.java |  322 --
 .../nifi/processors/standard/util/FileInfo.java |  167 -
 .../processors/standard/util/FileTransfer.java  |  244 --
 .../standard/util/FlowFileSessionWrapper.java   |   44 -
 .../standard/util/NLKBufferedReader.java        |  187 -
 .../standard/util/SFTPConnection.java           |   82 -
 .../processors/standard/util/SFTPTransfer.java  |  637 ---
 .../processors/standard/util/SFTPUtils.java     |  324 --
 .../standard/util/UDPStreamConsumer.java        |  214 --
 .../util/ValidatingBase64InputStream.java       |   76 -
 .../standard/util/XmlElementNotifier.java       |   22 -
 .../standard/util/XmlSplitterSaxParser.java     |  112 -
 .../org.apache.nifi.processor.Processor         |   58 -
 .../index.html                                  |   63 -
 .../index.html                                  |  166 -
 .../index.html                                  |  116 -
 .../index.html                                  |   65 -
 .../index.html                                  |  147 -
 .../index.html                                  |  106 -
 .../index.html                                  |   97 -
 .../index.html                                  |  160 -
 .../index.html                                  |  135 -
 .../index.html                                  |  311 --
 .../index.html                                  |  111 -
 .../index.html                                  |   64 -
 .../index.html                                  |  227 --
 .../index.html                                  |  186 -
 .../index.html                                  |  143 -
 .../index.html                                  |  250 --
 .../index.html                                  |   88 -
 .../index.html                                  |   89 -
 .../index.html                                  |  136 -
 .../index.html                                  |  181 -
 .../index.html                                  |   86 -
 .../index.html                                  |  144 -
 .../index.html                                  |   80 -
 .../index.html                                  |  347 --
 .../index.html                                  |   64 -
 .../index.html                                  |  143 -
 .../index.html                                  |  187 -
 .../index.html                                  |  114 -
 .../index.html                                  |  283 --
 .../index.html                                  |  109 -
 .../index.html                                  |  281 --
 .../index.html                                  |   91 -
 .../index.html                                  |  114 -
 .../index.html                                  |  110 -
 .../index.html                                  |   82 -
 .../index.html                                  |   85 -
 .../index.html                                  |  100 -
 .../index.html                                  |  123 -
 .../index.html                                  |  107 -
 .../index.html                                  |  121 -
 .../index.html                                  |   64 -
 .../index.html                                  |   63 -
 .../index.html                                  |  163 -
 .../index.html                                  |   56 -
 .../src/test/java/TestIngestAndUpdate.java      |   34 -
 .../src/test/java/TestSuccess.java              |   24 -
 .../processors/standard/HelloWorldServlet.java  |   41 -
 .../standard/RESTServiceContentModified.java    |   78 -
 .../standard/TestBase64EncodeContent.java       |   83 -
 .../standard/TestCompressContent.java           |  111 -
 .../processors/standard/TestControlRate.java    |   68 -
 .../standard/TestConvertCharacterSet.java       |   47 -
 .../standard/TestDetectDuplicate.java           |  206 -
 .../processors/standard/TestDistributeLoad.java |  139 -
 .../processors/standard/TestEncryptContent.java |   65 -
 .../standard/TestEvaluateRegularExpression.java |  319 --
 .../processors/standard/TestEvaluateXPath.java  |  159 -
 .../processors/standard/TestEvaluateXQuery.java |  651 ----
 .../standard/TestExecuteStreamCommand.java      |  188 -
 .../nifi/processors/standard/TestGetFile.java   |  186 -
 .../nifi/processors/standard/TestGetHTTP.java   |  354 --
 .../processors/standard/TestHashAttribute.java  |   99 -
 .../processors/standard/TestHashContent.java    |   67 -
 .../standard/TestIdentifyMimeType.java          |  131 -
 .../processors/standard/TestInvokeHTTP.java     |  593 ---
 .../nifi/processors/standard/TestListenUDP.java |  214 --
 .../processors/standard/TestMergeContent.java   |  593 ---
 .../processors/standard/TestModifyBytes.java    |  190 -
 .../standard/TestMonitorActivity.java           |  193 -
 .../processors/standard/TestReplaceText.java    |  371 --
 .../standard/TestReplaceTextLineByLine.java     |  337 --
 .../standard/TestReplaceTextWithMapping.java    |  332 --
 .../standard/TestRouteOnAttribute.java          |  161 -
 .../processors/standard/TestRouteOnContent.java |   73 -
 .../processors/standard/TestScanAttribute.java  |  149 -
 .../processors/standard/TestScanContent.java    |   85 -
 .../processors/standard/TestSegmentContent.java |   65 -
 .../nifi/processors/standard/TestServer.java    |  177 -
 .../processors/standard/TestSplitContent.java   |  235 --
 .../nifi/processors/standard/TestSplitText.java |  173 -
 .../nifi/processors/standard/TestSplitXml.java  |  103 -
 .../processors/standard/TestTransformXml.java   |  120 -
 .../processors/standard/TestUnpackContent.java  |  226 --
 .../processors/standard/TestValidateXml.java    |   42 -
 .../standard/UserAgentTestingServlet.java       |   42 -
 .../CharacterSetConversionSamples/Converted.txt |    1 -
 .../Converted2.txt                              |  Bin 134580 -> 0 bytes
 .../CharacterSetConversionSamples/Original.txt  |  418 --
 .../resources/CompressedData/SampleFile.txt     |  418 --
 .../resources/CompressedData/SampleFile.txt.bz2 |  Bin 388 -> 0 bytes
 .../resources/CompressedData/SampleFile.txt.gz  |  Bin 321 -> 0 bytes
 .../CompressedData/SampleFile1.txt.bz2          |  Bin 154 -> 0 bytes
 .../resources/CompressedData/SampleFile1.txt.gz |  Bin 321 -> 0 bytes
 .../CompressedData/SampleFileConcat.txt         |  835 ----
 .../CompressedData/SampleFileConcat.txt.bz2     |  Bin 542 -> 0 bytes
 .../test/resources/ExecuteCommand/1000bytes.txt |    1 -
 .../ExecuteCommand/TestIngestAndUpdate.jar      |  Bin 1170 -> 0 bytes
 .../resources/ExecuteCommand/TestSuccess.jar    |  Bin 827 -> 0 bytes
 .../src/test/resources/ExecuteCommand/test.txt  |    2 -
 .../dictionary-with-empty-new-lines             |    5 -
 .../ScanAttribute/dictionary-with-extra-info    |    9 -
 .../test/resources/ScanAttribute/dictionary1    |    5 -
 .../test/resources/TestIdentifyMimeType/1.7z    |  Bin 133 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.jar   |  Bin 466 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.mdb   |  Bin 237568 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.pdf   |  Bin 277032 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.tar   |  Bin 2048 -> 0 bytes
 .../resources/TestIdentifyMimeType/1.txt.bz2    |  Bin 59 -> 0 bytes
 .../resources/TestIdentifyMimeType/1.txt.gz     |  Bin 50 -> 0 bytes
 .../test/resources/TestIdentifyMimeType/1.xml   |   20 -
 .../test/resources/TestIdentifyMimeType/1.zip   |  Bin 165 -> 0 bytes
 .../TestIdentifyMimeType/bgBannerFoot.png       |  Bin 189 -> 0 bytes
 .../TestIdentifyMimeType/blueBtnBg.jpg          |  Bin 356 -> 0 bytes
 .../TestIdentifyMimeType/flowfilev1.tar         |  Bin 10240 -> 0 bytes
 .../resources/TestIdentifyMimeType/flowfilev3   |  Bin 40 -> 0 bytes
 .../resources/TestIdentifyMimeType/grid.gif     |  Bin 135 -> 0 bytes
 .../test/resources/TestMergeContent/demarcate   |    1 -
 .../src/test/resources/TestMergeContent/foot    |    1 -
 .../src/test/resources/TestMergeContent/head    |    1 -
 .../test/resources/TestModifyBytes/noFooter.txt |   10 -
 .../TestModifyBytes/noFooter_noHeader.txt       |   10 -
 .../test/resources/TestModifyBytes/noHeader.txt |   11 -
 .../test/resources/TestModifyBytes/testFile.txt |   11 -
 .../TestReplaceTextLineByLine/$1$1.txt          |   11 -
 .../BRue_cRue_RiRey.txt                         |   11 -
 .../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt |   11 -
 .../TestReplaceTextLineByLine/D$d_h$d.txt       |   11 -
 .../TestReplaceTextLineByLine/Good.txt          |    1 -
 .../TestReplaceTextLineByLine/Spider.txt        |   11 -
 .../TestReplaceTextLineByLine/[DODO].txt        |   11 -
 .../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt |   11 -
 .../TestReplaceTextLineByLine/cu_Po.txt         |   11 -
 .../TestReplaceTextLineByLine/food.txt          |   11 -
 .../TestReplaceTextLineByLine/testFile.txt      |   11 -
 .../color-fruit-backreference-mapping.txt       |    7 -
 .../color-fruit-blank-mapping.txt               |    7 -
 .../color-fruit-escaped-dollar-mapping.txt      |    7 -
 ...t-excessive-backreference-mapping-simple.txt |    6 -
 ...or-fruit-excessive-backreference-mapping.txt |    6 -
 ...olor-fruit-invalid-backreference-mapping.txt |    7 -
 .../color-fruit-mapping.txt                     |    7 -
 .../color-fruit-no-match-mapping.txt            |    7 -
 .../color-fruit-space-mapping.txt               |    6 -
 .../colors-without-dashes.txt                   |    4 -
 .../TestReplaceTextWithMapping/colors.txt       |    4 -
 .../test/resources/TestScanContent/helloWorld   |    1 -
 .../resources/TestScanContent/wellthengood-bye  |    1 -
 .../src/test/resources/TestSplitText/1.txt      |    5 -
 .../src/test/resources/TestSplitText/2.txt      |    5 -
 .../src/test/resources/TestSplitText/3.txt      |    5 -
 .../src/test/resources/TestSplitText/4.txt      |    3 -
 .../src/test/resources/TestSplitText/5.txt      |    7 -
 .../src/test/resources/TestSplitText/6.txt      |    7 -
 .../test/resources/TestSplitText/original.txt   |   12 -
 .../test/resources/TestTransformXml/math.html   |    8 -
 .../test/resources/TestTransformXml/math.xml    |   21 -
 .../test/resources/TestTransformXml/math.xsl    |   36 -
 .../test/resources/TestTransformXml/tokens.csv  |    2 -
 .../test/resources/TestTransformXml/tokens.xml  |   17 -
 .../test/resources/TestTransformXml/tokens.xsl  |  103 -
 .../resources/TestUnpackContent/data.flowfilev2 |  Bin 255 -> 0 bytes
 .../resources/TestUnpackContent/data.flowfilev3 |  Bin 357 -> 0 bytes
 .../test/resources/TestUnpackContent/data.tar   |  Bin 3584 -> 0 bytes
 .../test/resources/TestUnpackContent/data.zip   |  Bin 359 -> 0 bytes
 .../resources/TestUnpackContent/folder/cal.txt  |    8 -
 .../resources/TestUnpackContent/folder/date.txt |    1 -
 .../src/test/resources/TestXml/XmlBundle.xsd    |   34 -
 .../src/test/resources/TestXml/fruit.xml        |   47 -
 .../src/test/resources/TestXml/subNode.xml      |   21 -
 .../src/test/resources/TestXml/xml-bundle-1     |   51 -
 .../src/test/resources/TestXml/xml-snippet.xml  |   25 -
 .../src/test/resources/hello.txt                |    1 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../src/test/resources/localhost.cer            |  Bin 829 -> 0 bytes
 .../src/test/resources/logback-test.xml         |   49 -
 .../standard-reporting-tasks/pom.xml            |   60 -
 .../ControllerStatusReportingTask.java          |  347 --
 .../nifi/controller/MonitorDiskUsage.java       |  113 -
 .../apache/nifi/controller/MonitorMemory.java   |  260 --
 .../org.apache.nifi.reporting.ReportingTask     |   17 -
 .../index.html                                  |   85 -
 .../index.html                                  |   58 -
 .../index.html                                  |   77 -
 .../nifi/controller/TestMonitorDiskUsage.java   |   61 -
 .../pom.xml                                     |   36 -
 .../distributed/cache/client/Deserializer.java  |   41 -
 .../cache/client/DistributedMapCacheClient.java |  114 -
 .../cache/client/DistributedSetCacheClient.java |   74 -
 .../distributed/cache/client/Serializer.java    |   40 -
 .../exception/DeserializationException.java     |   33 -
 .../exception/SerializationException.java       |   33 -
 .../distributed-cache-client-service/pom.xml    |   56 -
 .../distributed/cache/client/CommsSession.java  |   46 -
 .../DistributedMapCacheClientService.java       |  305 --
 .../DistributedSetCacheClientService.java       |  215 --
 .../cache/client/SSLCommsSession.java           |  106 -
 .../cache/client/StandardCommsSession.java      |  124 -
 ...org.apache.nifi.controller.ControllerService |   16 -
 .../index.html                                  |   78 -
 .../index.html                                  |   51 -
 .../distributed-cache-protocol/pom.xml          |   39 -
 .../cache/protocol/ProtocolHandshake.java       |  119 -
 .../protocol/exception/HandshakeException.java  |   27 -
 .../distributed-cache-server/pom.xml            |   75 -
 .../cache/server/AbstractCacheServer.java       |  199 -
 .../distributed/cache/server/CacheRecord.java   |   57 -
 .../distributed/cache/server/CacheServer.java   |   26 -
 .../cache/server/DistributedCacheServer.java    |  107 -
 .../cache/server/DistributedSetCacheServer.java |   68 -
 .../cache/server/EvictionPolicy.java            |   73 -
 .../cache/server/SetCacheServer.java            |  104 -
 .../server/map/DistributedMapCacheServer.java   |   71 -
 .../distributed/cache/server/map/MapCache.java  |   29 -
 .../cache/server/map/MapCacheRecord.java        |   58 -
 .../cache/server/map/MapCacheServer.java        |  145 -
 .../cache/server/map/MapPutResult.java          |   59 -
 .../cache/server/map/PersistentMapCache.java    |  210 -
 .../cache/server/map/SimpleMapCache.java        |  165 -
 .../cache/server/set/PersistentSetCache.java    |  194 -
 .../distributed/cache/server/set/SetCache.java  |   29 -
 .../cache/server/set/SetCacheRecord.java        |   55 -
 .../cache/server/set/SetCacheResult.java        |   43 -
 .../cache/server/set/SimpleSetCache.java        |  117 -
 ...org.apache.nifi.controller.ControllerService |   16 -
 .../index.html                                  |   82 -
 .../cache/server/TestServerAndClient.java       |  535 ---
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../distributed-cache-services-nar/pom.xml      |   47 -
 .../distributed-cache-services-bundle/pom.xml   |   35 -
 .../load-distribution-service-api/pom.xml       |   37 -
 .../nifi/loading/LoadDistributionListener.java  |   24 -
 .../nifi/loading/LoadDistributionService.java   |   33 -
 nar-bundles/standard-services/pom.xml           |   39 -
 .../ssl-context-bundle/nar/pom.xml              |   39 -
 .../ssl-context-bundle/pom.xml                  |   34 -
 .../ssl-context-service/pom.xml                 |   52 -
 .../nifi/ssl/StandardSSLContextService.java     |  354 --
 ...org.apache.nifi.controller.ControllerService |   15 -
 .../index.html                                  |   63 -
 .../apache/nifi/ssl/SSLContextServiceTest.java  |  197 -
 .../java/org/apache/nifi/ssl/TestProcessor.java |   47 -
 .../src/test/resources/localhost-ks.jks         |  Bin 1298 -> 0 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 891 -> 0 bytes
 .../ssl-context-service-api/pom.xml             |   36 -
 .../org/apache/nifi/ssl/SSLContextService.java  |   55 -
 .../standard-services-api-nar/pom.xml           |   45 -
 .../update-attribute-bundle/model/pom.xml       |   27 -
 .../apache/nifi/update/attributes/Action.java   |   55 -
 .../nifi/update/attributes/Condition.java       |   49 -
 .../apache/nifi/update/attributes/Criteria.java |  141 -
 .../nifi/update/attributes/FlowFilePolicy.java  |   34 -
 .../org/apache/nifi/update/attributes/Rule.java |   65 -
 .../update/attributes/serde/CriteriaSerDe.java  |  128 -
 nar-bundles/update-attribute-bundle/nar/pom.xml |   48 -
 nar-bundles/update-attribute-bundle/pom.xml     |   56 -
 .../update-attribute-bundle/processor/pom.xml   |   54 -
 .../processors/attributes/UpdateAttribute.java  |  508 ---
 .../org.apache.nifi.processor.Processor         |   15 -
 .../index.html                                  |  253 --
 .../update/attributes/TestUpdateAttribute.java  |  425 --
 .../ui/nb-configuration.xml                     |   18 -
 nar-bundles/update-attribute-bundle/ui/pom.xml  |   91 -
 .../attributes/UpdateAttributeModelFactory.java |  127 -
 .../attributes/api/ObjectMapperResolver.java    |   51 -
 .../update/attributes/api/RuleResource.java     |  684 ----
 .../nifi/update/attributes/dto/ActionDTO.java   |   68 -
 .../update/attributes/dto/ConditionDTO.java     |   59 -
 .../nifi/update/attributes/dto/DtoFactory.java  |   68 -
 .../nifi/update/attributes/dto/RuleDTO.java     |   65 -
 .../update/attributes/entity/ActionEntity.java  |   65 -
 .../attributes/entity/ConditionEntity.java      |   65 -
 .../entity/EvaluationContextEntity.java         |   74 -
 .../update/attributes/entity/RuleEntity.java    |   65 -
 .../update/attributes/entity/RulesEntity.java   |   66 -
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 -
 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp   |  188 -
 .../ui/src/main/webapp/WEB-INF/web.xml          |   46 -
 .../ui/src/main/webapp/css/main.css             |  426 ---
 .../ui/src/main/webapp/images/bgInputText.png   |  Bin 139 -> 0 bytes
 .../ui/src/main/webapp/images/buttonNew.png     |  Bin 590 -> 0 bytes
 .../ui/src/main/webapp/images/iconDelete.png    |  Bin 670 -> 0 bytes
 .../ui/src/main/webapp/images/iconInfo.png      |  Bin 550 -> 0 bytes
 .../ui/src/main/webapp/js/application.js        | 1915 ---------
 .../nar/pom.xml                                 |   36 -
 .../pom.xml                                     |   44 -
 .../volatile-provenance-repository/pom.xml      |   47 -
 .../VolatileProvenanceRepository.java           |  745 ----
 ...he.nifi.provenance.ProvenanceEventRepository |   15 -
 .../TestVolatileProvenanceRepository.java       |  178 -
 .../src/test/resources/nifi.properties          |   18 -
 nar-maven-plugin/pom.xml                        |  385 --
 .../src/main/java/nifi/NarMojo.java             |  613 ---
 .../resources/META-INF/plexus/components.xml    |   52 -
 nifi-api/.gitignore                             |    2 -
 nifi-api/pom.xml                                |   31 -
 .../apache/nifi/authorization/Authority.java    |   97 -
 .../nifi/authorization/AuthorityProvider.java   |  181 -
 .../AuthorityProviderConfigurationContext.java  |   52 -
 .../AuthorityProviderInitializationContext.java |   27 -
 .../authorization/AuthorityProviderLookup.java  |   25 -
 .../authorization/DownloadAuthorization.java    |   86 -
 .../annotation/AuthorityProviderContext.java    |   36 -
 .../exception/AuthorityAccessException.java     |   33 -
 .../IdentityAlreadyExistsException.java         |   32 -
 .../exception/ProviderCreationException.java    |   40 -
 .../exception/ProviderDestructionException.java |   40 -
 .../exception/UnknownIdentityException.java     |   32 -
 .../AbstractConfigurableComponent.java          |  221 --
 .../apache/nifi/components/AllowableValue.java  |  136 -
 .../nifi/components/ConfigurableComponent.java  |   77 -
 .../nifi/components/PropertyDescriptor.java     |  552 ---
 .../apache/nifi/components/PropertyValue.java   |  189 -
 .../nifi/components/ValidationContext.java      |   82 -
 .../nifi/components/ValidationResult.java       |  173 -
 .../org/apache/nifi/components/Validator.java   |   55 -
 .../controller/AbstractControllerService.java   |   91 -
 .../nifi/controller/ConfigurationContext.java   |   46 -
 .../nifi/controller/ControllerService.java      |  182 -
 .../ControllerServiceInitializationContext.java |   36 -
 .../controller/ControllerServiceLookup.java     |   66 -
 .../apache/nifi/controller/FlowFileQueue.java   |  200 -
 .../apache/nifi/controller/ScheduledState.java  |   37 -
 .../org/apache/nifi/controller/Snippet.java     |  113 -
 .../org/apache/nifi/controller/Triggerable.java |  116 -
 .../controller/annotation/OnConfigured.java     |   41 -
 .../repository/ContentRepository.java           |  314 --
 .../controller/repository/FlowFileRecord.java   |   54 -
 .../repository/FlowFileRepository.java          |  128 -
 .../repository/FlowFileSwapManager.java         |   63 -
 .../controller/repository/QueueProvider.java    |   35 -
 .../controller/repository/RepositoryRecord.java |  100 -
 .../repository/RepositoryRecordType.java        |   26 -
 .../repository/claim/ContentClaim.java          |   55 -
 .../repository/claim/ContentClaimManager.java   |  143 -
 .../controller/status/ConnectionStatus.java     |  193 -
 .../nifi/controller/status/PortStatus.java      |  201 -
 .../controller/status/ProcessGroupStatus.java   |  547 ---
 .../nifi/controller/status/ProcessorStatus.java |  275 --
 .../status/RemoteProcessGroupStatus.java        |  217 --
 .../nifi/controller/status/RunStatus.java       |   28 -
 .../controller/status/TransmissionStatus.java   |   23 -
 .../history/ComponentStatusRepository.java      |  167 -
 .../status/history/MetricDescriptor.java        |   75 -
 .../status/history/StatusHistory.java           |   50 -
 .../status/history/StatusSnapshot.java          |   48 -
 .../controller/status/history/ValueMapper.java  |   23 -
 .../controller/status/history/ValueReducer.java |   25 -
 .../org/apache/nifi/events/EventReporter.java   |   27 -
 .../nifi/expression/AttributeExpression.java    |   79 -
 .../expression/AttributeValueDecorator.java     |   29 -
 .../expression/ExpressionLanguageCompiler.java  |   69 -
 .../java/org/apache/nifi/flowfile/FlowFile.java |  109 -
 .../nifi/flowfile/FlowFilePrioritizer.java      |   30 -
 .../java/org/apache/nifi/logging/LogLevel.java  |   27 -
 .../org/apache/nifi/logging/ProcessorLog.java   |   73 -
 .../nifi/processor/AbstractProcessor.java       |   37 -
 .../AbstractSessionFactoryProcessor.java        |  122 -
 .../org/apache/nifi/processor/DataUnit.java     |  248 --
 .../apache/nifi/processor/FlowFileFilter.java   |   91 -
 .../apache/nifi/processor/ProcessContext.java   |  132 -
 .../apache/nifi/processor/ProcessSession.java   |  713 ----
 .../nifi/processor/ProcessSessionFactory.java   |   30 -
 .../org/apache/nifi/processor/Processor.java    |   91 -
 .../ProcessorInitializationContext.java         |   53 -
 .../org/apache/nifi/processor/QueueSize.java    |   49 -
 .../org/apache/nifi/processor/Relationship.java |  127 -
 .../nifi/processor/SchedulingContext.java       |   70 -
 .../annotation/CapabilityDescription.java       |   39 -
 .../nifi/processor/annotation/EventDriven.java  |   49 -
 .../nifi/processor/annotation/OnAdded.java      |   41 -
 .../nifi/processor/annotation/OnRemoved.java    |   42 -
 .../nifi/processor/annotation/OnScheduled.java  |   44 -
 .../nifi/processor/annotation/OnShutdown.java   |   38 -
 .../nifi/processor/annotation/OnStopped.java    |   56 -
 .../processor/annotation/OnUnscheduled.java     |   46 -
 .../processor/annotation/SideEffectFree.java    |   45 -
 .../processor/annotation/SupportsBatching.java  |   51 -
 .../apache/nifi/processor/annotation/Tags.java  |   44 -
 .../processor/annotation/TriggerSerially.java   |   39 -
 .../TriggerWhenAnyDestinationAvailable.java     |   40 -
 .../processor/annotation/TriggerWhenEmpty.java  |   41 -
 .../exception/FlowFileAccessException.java      |   37 -
 .../exception/FlowFileHandlingException.java    |   39 -
 .../exception/MissingFlowFileException.java     |   37 -
 .../processor/exception/ProcessException.java   |   44 -
 .../nifi/processor/io/InputStreamCallback.java  |   38 -
 .../nifi/processor/io/OutputStreamCallback.java |   39 -
 .../nifi/processor/io/StreamCallback.java       |   41 -
 .../nifi/provenance/ProvenanceEventBuilder.java |  308 --
 .../nifi/provenance/ProvenanceEventRecord.java  |  296 --
 .../provenance/ProvenanceEventRepository.java   |  190 -
 .../nifi/provenance/ProvenanceEventType.java    |   92 -
 .../nifi/provenance/ProvenanceReporter.java     |  616 ---
 .../lineage/ComputeLineageResult.java           |   70 -
 .../lineage/ComputeLineageSubmission.java       |   82 -
 .../apache/nifi/provenance/lineage/Lineage.java |   41 -
 .../lineage/LineageComputationType.java         |   27 -
 .../nifi/provenance/lineage/LineageEdge.java    |   26 -
 .../nifi/provenance/lineage/LineageNode.java    |   61 -
 .../provenance/lineage/LineageNodeType.java     |   23 -
 .../lineage/ProvenanceEventLineageNode.java     |   32 -
 .../apache/nifi/provenance/search/Query.java    |   91 -
 .../nifi/provenance/search/QueryResult.java     |   77 -
 .../nifi/provenance/search/QuerySubmission.java |   63 -
 .../nifi/provenance/search/SearchTerm.java      |   24 -
 .../nifi/provenance/search/SearchTerms.java     |   39 -
 .../nifi/provenance/search/SearchableField.java |   62 -
 .../provenance/search/SearchableFieldType.java  |   28 -
 .../nifi/reporting/AbstractReportingTask.java   |   94 -
 .../org/apache/nifi/reporting/Bulletin.java     |  120 -
 .../apache/nifi/reporting/BulletinQuery.java    |  109 -
 .../nifi/reporting/BulletinRepository.java      |   88 -
 .../org/apache/nifi/reporting/EventAccess.java  |   53 -
 .../nifi/reporting/InitializationException.java |   32 -
 .../apache/nifi/reporting/ReportingContext.java |   99 -
 .../ReportingInitializationContext.java         |   80 -
 .../apache/nifi/reporting/ReportingTask.java    |   75 -
 .../org/apache/nifi/reporting/Severity.java     |   24 -
 .../nifi/scheduling/SchedulingStrategy.java     |   86 -
 .../org/apache/nifi/search/SearchContext.java   |   57 -
 .../org/apache/nifi/search/SearchResult.java    |   82 -
 .../java/org/apache/nifi/search/Searchable.java |   27 -
 .../nifi/web/ClusterRequestException.java       |   38 -
 .../nifi/web/InvalidRevisionException.java      |   33 -
 .../org/apache/nifi/web/NiFiWebContext.java     |  121 -
 .../apache/nifi/web/NiFiWebContextConfig.java   |   54 -
 .../nifi/web/ProcessorConfigurationAction.java  |  137 -
 .../java/org/apache/nifi/web/ProcessorInfo.java |  110 -
 .../nifi/web/ResourceNotFoundException.java     |   32 -
 .../main/java/org/apache/nifi/web/Revision.java |  110 -
 .../nifi/components/TestPropertyDescriptor.java |   59 -
 .../org/apache/nifi/processor/TestDataUnit.java |   44 -
 nifi-api/src/test/resources/logback-test.xml    |   27 -
 nifi-bootstrap/pom.xml                          |   29 -
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   92 -
 .../org/apache/nifi/bootstrap/NiFiListener.java |  128 -
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  941 -----
 .../org/apache/nifi/bootstrap/ShutdownHook.java |   86 -
 .../exception/InvalidCommandException.java      |   37 -
 .../bootstrap/util/LimitingInputStream.java     |  107 -
 nifi-docs/pom.xml                               |  131 -
 .../src/main/asciidoc/administration-guide.adoc |  160 -
 .../src/main/asciidoc/developer-guide.adoc      |   45 -
 .../asciidoc/expression-language-guide.adoc     | 1727 ---------
 .../images/add-processor-with-tag-cloud.png     |  Bin 22925 -> 0 bytes
 .../src/main/asciidoc/images/add-processor.png  |  Bin 31524 -> 0 bytes
 .../src/main/asciidoc/images/addConnect.png     |  Bin 1996 -> 0 bytes
 .../src/main/asciidoc/images/comments-tab.png   |  Bin 6431 -> 0 bytes
 .../src/main/asciidoc/images/components.png     |  Bin 8992 -> 0 bytes
 .../asciidoc/images/connection-settings.png     |  Bin 15214 -> 0 bytes
 .../main/asciidoc/images/create-connection.png  |  Bin 9430 -> 0 bytes
 .../asciidoc/images/edit-property-dropdown.png  |  Bin 14291 -> 0 bytes
 .../asciidoc/images/edit-property-textarea.png  |  Bin 17146 -> 0 bytes
 .../main/asciidoc/images/event-attributes.png   |  Bin 94668 -> 0 bytes
 .../src/main/asciidoc/images/event-content.png  |  Bin 88726 -> 0 bytes
 .../src/main/asciidoc/images/event-details.png  |  Bin 139736 -> 0 bytes
 .../src/main/asciidoc/images/expand-event.png   |  Bin 40476 -> 0 bytes
 .../main/asciidoc/images/expanded-events.png    |  Bin 76082 -> 0 bytes
 .../src/main/asciidoc/images/find-parents.png   |  Bin 35831 -> 0 bytes
 .../src/main/asciidoc/images/iconAlert.png      |  Bin 1396 -> 0 bytes
 .../src/main/asciidoc/images/iconConnection.png |  Bin 1517 -> 0 bytes
 .../src/main/asciidoc/images/iconDelete.png     |  Bin 670 -> 0 bytes
 .../src/main/asciidoc/images/iconDisable.png    |  Bin 764 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconEdit.png |  Bin 493 -> 0 bytes
 .../src/main/asciidoc/images/iconEnable.png     |  Bin 667 -> 0 bytes
 .../src/main/asciidoc/images/iconExport.png     |  Bin 453 -> 0 bytes
 .../main/asciidoc/images/iconFlowHistory.png    |  Bin 2463 -> 0 bytes
 .../src/main/asciidoc/images/iconFunnel.png     |  Bin 1223 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconInfo.png |  Bin 550 -> 0 bytes
 .../src/main/asciidoc/images/iconInputPort.png  |  Bin 1842 -> 0 bytes
 .../main/asciidoc/images/iconInputPortSmall.png |  Bin 532 -> 0 bytes
 .../src/main/asciidoc/images/iconLabel.png      |  Bin 838 -> 0 bytes
 .../src/main/asciidoc/images/iconLineage.png    |  Bin 2214 -> 0 bytes
 .../main/asciidoc/images/iconNewTemplate.png    |  Bin 557 -> 0 bytes
 .../src/main/asciidoc/images/iconNotSecure.png  |  Bin 221 -> 0 bytes
 .../src/main/asciidoc/images/iconOutputPort.png |  Bin 1658 -> 0 bytes
 .../asciidoc/images/iconOutputPortSmall.png     |  Bin 459 -> 0 bytes
 .../main/asciidoc/images/iconProcessGroup.png   |  Bin 1422 -> 0 bytes
 .../src/main/asciidoc/images/iconProcessor.png  |  Bin 1446 -> 0 bytes
 .../src/main/asciidoc/images/iconProvenance.png |  Bin 2268 -> 0 bytes
 .../asciidoc/images/iconRemoteProcessGroup.png  |  Bin 674 -> 0 bytes
 .../src/main/asciidoc/images/iconResize.png     |  Bin 165 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconRun.png  |  Bin 538 -> 0 bytes
 .../src/main/asciidoc/images/iconSecure.png     |  Bin 225 -> 0 bytes
 .../src/main/asciidoc/images/iconSettings.png   |  Bin 2638 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/iconStop.png |  Bin 402 -> 0 bytes
 .../src/main/asciidoc/images/iconSummary.png    |  Bin 272 -> 0 bytes
 .../src/main/asciidoc/images/iconTemplate.png   |  Bin 970 -> 0 bytes
 .../asciidoc/images/iconTransmissionActive.png  |  Bin 1330 -> 0 bytes
 .../images/iconTransmissionInactive.png         |  Bin 1248 -> 0 bytes
 .../src/main/asciidoc/images/iconUsers.png      |  Bin 2272 -> 0 bytes
 .../main/asciidoc/images/iconViewDetails.png    |  Bin 1788 -> 0 bytes
 .../images/instantiate-template-description.png |  Bin 6508 -> 0 bytes
 .../asciidoc/images/instantiate-template.png    |  Bin 3216 -> 0 bytes
 .../main/asciidoc/images/invalid-processor.png  |  Bin 8187 -> 0 bytes
 .../main/asciidoc/images/lineage-flowfile.png   |  Bin 3855 -> 0 bytes
 .../asciidoc/images/lineage-graph-annotated.png |  Bin 170122 -> 0 bytes
 nifi-docs/src/main/asciidoc/images/new-flow.png |  Bin 266913 -> 0 bytes
 .../main/asciidoc/images/nifi-arch-cluster.png  |  Bin 50384 -> 0 bytes
 .../src/main/asciidoc/images/nifi-arch.png      |  Bin 43509 -> 0 bytes
 .../main/asciidoc/images/nifi-navigation.png    |  Bin 339195 -> 0 bytes
 .../asciidoc/images/nifi-toolbar-components.png |  Bin 262314 -> 0 bytes
 .../src/main/asciidoc/images/parent-found.png   |  Bin 42814 -> 0 bytes
 .../asciidoc/images/process-group-anatomy.png   |  Bin 48554 -> 0 bytes
 .../main/asciidoc/images/processor-anatomy.png  |  Bin 37035 -> 0 bytes
 .../images/processor-connection-bubble.png      |  Bin 9936 -> 0 bytes
 .../src/main/asciidoc/images/properties-tab.png |  Bin 11847 -> 0 bytes
 .../asciidoc/images/provenance-annotated.png    |  Bin 530929 -> 0 bytes
 .../main/asciidoc/images/provenance-table.png   |  Bin 466599 -> 0 bytes
 .../asciidoc/images/remote-group-anatomy.png    |  Bin 61383 -> 0 bytes
 .../images/remote-group-ports-dialog.png        |  Bin 21662 -> 0 bytes
 .../images/remote-port-connection-status.png    |  Bin 57332 -> 0 bytes
 .../src/main/asciidoc/images/scheduling-tab.png |  Bin 12288 -> 0 bytes
 .../src/main/asciidoc/images/search-events.png  |  Bin 62626 -> 0 bytes
 .../images/search-receive-event-abc.png         |  Bin 67817 -> 0 bytes
 .../src/main/asciidoc/images/settings-tab.png   |  Bin 16225 -> 0 bytes
 .../src/main/asciidoc/images/simple-flow.png    |  Bin 196808 -> 0 bytes
 .../src/main/asciidoc/images/stats-history.png  |  Bin 33205 -> 0 bytes
 .../src/main/asciidoc/images/status-bar.png     |  Bin 249077 -> 0 bytes
 .../main/asciidoc/images/summary-annotated.png  |  Bin 111956 -> 0 bytes
 .../src/main/asciidoc/images/summary-table.png  |  Bin 62114 -> 0 bytes
 .../main/asciidoc/images/valid-processor.png    |  Bin 8090 -> 0 bytes
 nifi-docs/src/main/asciidoc/overview.adoc       |  296 --
 nifi-docs/src/main/asciidoc/user-guide.adoc     | 1286 -------
 nifi-docs/src/main/assembly/dependencies.xml    |   28 -
 nifi-mock/pom.xml                               |   54 -
 .../MockProvenanceEventRepository.java          |  131 -
 .../apache/nifi/reporting/BulletinFactory.java  |   43 -
 .../org/apache/nifi/reporting/MockBulletin.java |   24 -
 .../util/ControllerServiceConfiguration.java    |   74 -
 .../nifi/util/MockBulletinRepository.java       |   74 -
 .../nifi/util/MockConfigurationContext.java     |   50 -
 ...kControllerServiceInitializationContext.java |   41 -
 .../nifi/util/MockControllerServiceLookup.java  |   76 -
 .../org/apache/nifi/util/MockEventAccess.java   |   70 -
 .../java/org/apache/nifi/util/MockFlowFile.java |  279 --
 .../org/apache/nifi/util/MockFlowFileQueue.java |   85 -
 .../apache/nifi/util/MockProcessContext.java    |  284 --
 .../apache/nifi/util/MockProcessSession.java    | 1006 -----
 .../MockProcessorInitializationContext.java     |   74 -
 .../org/apache/nifi/util/MockProcessorLog.java  |  402 --
 .../org/apache/nifi/util/MockPropertyValue.java |  185 -
 .../nifi/util/MockProvenanceReporter.java       |  202 -
 .../apache/nifi/util/MockReportingContext.java  |  117 -
 .../MockReportingInitializationContext.java     |   81 -
 .../apache/nifi/util/MockSessionFactory.java    |   46 -
 .../apache/nifi/util/MockValidationContext.java |   93 -
 .../org/apache/nifi/util/ReflectionUtils.java   |  149 -
 .../apache/nifi/util/SharedSessionState.java    |   72 -
 .../apache/nifi/util/SingleSessionFactory.java  |   35 -
 .../nifi/util/StandardProcessorTestRunner.java  |  492 ---
 .../java/org/apache/nifi/util/TestRunner.java   |  542 ---
 .../java/org/apache/nifi/util/TestRunners.java  |   37 -
 nifi/assembly/pom.xml                           |  458 +++
 .../assembly/src/main/assembly/dependencies.xml |  140 +
 nifi/commons/data-provenance-utils/.gitignore   |    2 +
 nifi/commons/data-provenance-utils/pom.xml      |   40 +
 .../nifi/provenance/AsyncLineageSubmission.java |   87 +
 .../nifi/provenance/AsyncQuerySubmission.java   |   81 +
 .../nifi/provenance/NamedSearchableField.java   |   95 +
 .../nifi/provenance/SearchableFieldParser.java  |   53 +
 .../nifi/provenance/SearchableFields.java       |   84 +
 .../nifi/provenance/StandardLineageResult.java  |  324 ++
 .../StandardProvenanceEventRecord.java          |  752 ++++
 .../nifi/provenance/StandardQueryResult.java    |  168 +
 .../nifi/provenance/lineage/EdgeNode.java       |   74 +
 .../nifi/provenance/lineage/EventNode.java      |  109 +
 .../provenance/lineage/FlowFileLineage.java     |   76 +
 .../nifi/provenance/lineage/FlowFileNode.java   |   83 +
 nifi/commons/flowfile-packager/pom.xml          |   41 +
 .../org/apache/nifi/util/FlowFilePackager.java  |   28 +
 .../apache/nifi/util/FlowFilePackagerV1.java    |  104 +
 .../apache/nifi/util/FlowFilePackagerV2.java    |  146 +
 .../apache/nifi/util/FlowFilePackagerV3.java    |   93 +
 .../apache/nifi/util/FlowFileUnpackager.java    |   30 +
 .../apache/nifi/util/FlowFileUnpackagerV1.java  |  155 +
 .../apache/nifi/util/FlowFileUnpackagerV2.java  |  143 +
 .../apache/nifi/util/FlowFileUnpackagerV3.java  |  161 +
 .../java/org/apache/nifi/util/Unpackage.java    |  119 +
 .../nifi/util/TestPackageUnpackageV3.java       |   56 +
 nifi/commons/nifi-expression-language/pom.xml   |   56 +
 .../language/antlr/AttributeExpressionLexer.g   |  210 +
 .../language/antlr/AttributeExpressionParser.g  |  139 +
 .../output/AttributeExpressionLexer.tokens      |   88 +
 .../expression/language/EmptyPreparedQuery.java |   62 +
 .../language/InvalidPreparedQuery.java          |   71 +
 .../expression/language/PreparedQuery.java      |   39 +
 .../attribute/expression/language/Query.java    | 1186 ++++++
 .../language/StandardAttributeExpression.java   |   65 +
 .../StandardExpressionLanguageCompiler.java     |   58 +
 .../language/StandardPreparedQuery.java         |   83 +
 .../language/evaluation/BooleanEvaluator.java   |   32 +
 .../language/evaluation/BooleanQueryResult.java |   43 +
 .../language/evaluation/DateEvaluator.java      |   34 +
 .../language/evaluation/DateQueryResult.java    |   45 +
 .../language/evaluation/Evaluator.java          |   32 +
 .../language/evaluation/NumberEvaluator.java    |   33 +
 .../language/evaluation/NumberQueryResult.java  |   43 +
 .../language/evaluation/QueryResult.java        |   26 +
 .../language/evaluation/StringEvaluator.java    |   32 +
 .../language/evaluation/StringQueryResult.java  |   43 +
 .../evaluation/cast/BooleanCastEvaluator.java   |   50 +
 .../evaluation/cast/DateCastEvaluator.java      |  117 +
 .../evaluation/cast/NumberCastEvaluator.java    |   72 +
 .../evaluation/cast/StringCastEvaluator.java    |   49 +
 .../evaluation/functions/AndEvaluator.java      |   60 +
 .../evaluation/functions/AppendEvaluator.java   |   50 +
 .../functions/AttributeEvaluator.java           |   45 +
 .../evaluation/functions/ContainsEvaluator.java |   53 +
 .../functions/DateToNumberEvaluator.java        |   50 +
 .../evaluation/functions/DivideEvaluator.java   |   57 +
 .../evaluation/functions/EndsWithEvaluator.java |   53 +
 .../evaluation/functions/EqualsEvaluator.java   |   89 +
 .../functions/EqualsIgnoreCaseEvaluator.java    |   60 +
 .../evaluation/functions/FindEvaluator.java     |   72 +
 .../evaluation/functions/FormatEvaluator.java   |   61 +
 .../functions/GreaterThanEvaluator.java         |   60 +
 .../functions/GreaterThanOrEqualEvaluator.java  |   60 +
 .../evaluation/functions/HostnameEvaluator.java |   58 +
 .../evaluation/functions/IPEvaluator.java       |   46 +
 .../evaluation/functions/IndexOfEvaluator.java  |   53 +
 .../evaluation/functions/IsEmptyEvaluator.java  |   43 +
 .../evaluation/functions/IsNullEvaluator.java   |   45 +
 .../functions/LastIndexOfEvaluator.java         |   53 +
 .../evaluation/functions/LengthEvaluator.java   |   46 +
 .../evaluation/functions/LessThanEvaluator.java |   60 +
 .../functions/LessThanOrEqualEvaluator.java     |   60 +
 .../evaluation/functions/MatchesEvaluator.java  |   71 +
 .../evaluation/functions/MinusEvaluator.java    |   57 +
 .../evaluation/functions/ModEvaluator.java      |   57 +
 .../evaluation/functions/MultiplyEvaluator.java |   57 +
 .../evaluation/functions/NotEvaluator.java      |   49 +
 .../evaluation/functions/NotNullEvaluator.java  |   45 +
 .../evaluation/functions/NowEvaluator.java      |   39 +
 .../functions/NumberToDateEvaluator.java        |   52 +
 .../functions/OneUpSequenceEvaluator.java       |   41 +
 .../evaluation/functions/OrEvaluator.java       |   60 +
 .../evaluation/functions/PlusEvaluator.java     |   57 +
 .../evaluation/functions/PrependEvaluator.java  |   50 +
 .../functions/ReplaceAllEvaluator.java          |   55 +
 .../functions/ReplaceEmptyEvaluator.java        |   50 +
 .../evaluation/functions/ReplaceEvaluator.java  |   55 +
 .../functions/ReplaceNullEvaluator.java         |   47 +
 .../functions/StartsWithEvaluator.java          |   53 +
 .../functions/StringToDateEvaluator.java        |   65 +
 .../functions/SubstringAfterEvaluator.java      |   59 +
 .../functions/SubstringAfterLastEvaluator.java  |   55 +
 .../functions/SubstringBeforeEvaluator.java     |   58 +
 .../functions/SubstringBeforeLastEvaluator.java |   55 +
 .../functions/SubstringEvaluator.java           |   65 +
 .../evaluation/functions/ToLowerEvaluator.java  |   45 +
 .../evaluation/functions/ToNumberEvaluator.java |   46 +
 .../evaluation/functions/ToRadixEvaluator.java  |   77 +
 .../evaluation/functions/ToStringEvaluator.java |   45 +
 .../evaluation/functions/ToUpperEvaluator.java  |   45 +
 .../evaluation/functions/TrimEvaluator.java     |   45 +
 .../functions/UrlDecodeEvaluator.java           |   55 +
 .../functions/UrlEncodeEvaluator.java           |   55 +
 .../evaluation/functions/UuidEvaluator.java     |   39 +
 .../literals/BooleanLiteralEvaluator.java       |   44 +
 .../literals/NumberLiteralEvaluator.java        |   44 +
 .../literals/StringLiteralEvaluator.java        |   77 +
 .../evaluation/reduce/CountEvaluator.java       |   56 +
 .../evaluation/reduce/JoinEvaluator.java        |   59 +
 .../evaluation/reduce/ReduceEvaluator.java      |   23 +
 .../selection/AllAttributesEvaluator.java       |   68 +
 .../selection/AnyAttributeEvaluator.java        |   68 +
 .../AnyMatchingAttributeEvaluator.java          |   21 +
 .../selection/DelineatedAttributeEvaluator.java |   83 +
 .../evaluation/selection/MappingEvaluator.java  |   61 +
 .../selection/MultiAttributeEvaluator.java      |   24 +
 .../selection/MultiMatchAttributeEvaluator.java |   82 +
 .../selection/MultiNamedAttributeEvaluator.java |   64 +
 .../AttributeExpressionLanguageException.java   |   34 +
 ...ibuteExpressionLanguageParsingException.java |   34 +
 .../exception/IllegalAttributeException.java    |   29 +
 .../expression/language/TestQuery.java          | 1143 ++++++
 .../language/TestStandardPreparedQuery.java     |   92 +
 nifi/commons/nifi-logging-utils/pom.xml         |   36 +
 .../java/org/apache/nifi/logging/NiFiLog.java   |  367 ++
 nifi/commons/nifi-properties/.gitignore         |    3 +
 nifi/commons/nifi-properties/pom.xml            |   29 +
 .../org/apache/nifi/util/NiFiProperties.java    |  876 +++++
 .../java/org/apache/nifi/util/StringUtils.java  |   66 +
 nifi/commons/nifi-security-utils/pom.xml        |   40 +
 .../nifi/security/util/CertificateUtils.java    |  158 +
 .../nifi/security/util/EncryptionMethod.java    |   84 +
 .../apache/nifi/security/util/KeystoreType.java |   26 +
 .../nifi/security/util/SecurityStoreTypes.java  |  144 +
 .../nifi/security/util/SslContextFactory.java   |  180 +
 nifi/commons/nifi-socket-utils/pom.xml          |   60 +
 .../nifi/io/nio/AbstractChannelReader.java      |  166 +
 .../java/org/apache/nifi/io/nio/BufferPool.java |  114 +
 .../apache/nifi/io/nio/ChannelDispatcher.java   |  160 +
 .../org/apache/nifi/io/nio/ChannelListener.java |  228 ++
 .../nifi/io/nio/DatagramChannelReader.java      |   59 +
 .../apache/nifi/io/nio/SocketChannelReader.java |   55 +
 .../io/nio/consumer/AbstractStreamConsumer.java |  132 +
 .../nifi/io/nio/consumer/StreamConsumer.java    |   80 +
 .../io/nio/consumer/StreamConsumerFactory.java  |   27 +
 .../nifi/io/socket/SSLContextFactory.java       |  102 +
 .../io/socket/ServerSocketConfiguration.java    |   83 +
 .../nifi/io/socket/SocketConfiguration.java     |  116 +
 .../apache/nifi/io/socket/SocketListener.java   |  211 +
 .../org/apache/nifi/io/socket/SocketUtils.java  |  169 +
 .../socket/multicast/DiscoverableService.java   |   43 +
 .../multicast/DiscoverableServiceImpl.java      |   78 +
 .../multicast/MulticastConfiguration.java       |   99 +
 .../io/socket/multicast/MulticastListener.java  |  193 +
 .../multicast/MulticastServiceDiscovery.java    |   34 +
 .../multicast/MulticastServicesBroadcaster.java |   33 +
 .../socket/multicast/MulticastTimeToLive.java   |   50 +
 .../io/socket/multicast/MulticastUtils.java     |  109 +
 .../io/socket/multicast/ServiceDiscovery.java   |   31 +
 .../socket/multicast/ServicesBroadcaster.java   |   56 +
 .../apache/nifi/io/nio/example/ServerMain.java  |  141 +
 .../apache/nifi/io/nio/example/TCPClient.java   |   86 +
 .../apache/nifi/io/nio/example/UDPClient.java   |   51 +
 .../io/nio/example/UselessStreamConsumer.java   |   43 +
 .../src/test/resources/log4j.xml                |   36 +
 nifi/commons/nifi-utils/.gitignore              |    8 +
 nifi/commons/nifi-utils/pom.xml                 |   33 +
 .../flowfile/attributes/CoreAttributes.java     |   72 +
 .../attributes/FlowFileAttributeKey.java        |   21 +
 .../nifi/remote/StandardVersionNegotiator.java  |   81 +
 .../apache/nifi/remote/VersionNegotiator.java   |   65 +
 .../TransmissionDisabledException.java          |   25 +
 .../nifi/remote/io/CompressionInputStream.java  |  184 +
 .../nifi/remote/io/CompressionOutputStream.java |  147 +
 .../remote/io/InterruptableInputStream.java     |  117 +
 .../remote/io/InterruptableOutputStream.java    |   81 +
 .../remote/io/socket/BufferStateManager.java    |  111 +
 .../io/socket/SocketChannelInputStream.java     |  157 +
 .../io/socket/SocketChannelOutputStream.java    |  113 +
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  602 +++
 .../socket/ssl/SSLSocketChannelInputStream.java |   62 +
 .../ssl/SSLSocketChannelOutputStream.java       |   53 +
 .../nifi/stream/io/BufferedInputStream.java     |   37 +
 .../nifi/stream/io/BufferedOutputStream.java    |  140 +
 .../nifi/stream/io/ByteArrayInputStream.java    |  250 ++
 .../nifi/stream/io/ByteArrayOutputStream.java   |  250 ++
 .../nifi/stream/io/ByteCountingInputStream.java |  104 +
 .../stream/io/ByteCountingOutputStream.java     |   63 +
 .../apache/nifi/stream/io/DataOutputStream.java |  417 ++
 .../apache/nifi/stream/io/GZIPOutputStream.java |   41 +
 .../stream/io/LeakyBucketStreamThrottler.java   |  331 ++
 .../nifi/stream/io/NonCloseableInputStream.java |   56 +
 .../stream/io/NonCloseableOutputStream.java     |   51 +
 .../apache/nifi/stream/io/NullOutputStream.java |   46 +
 .../apache/nifi/stream/io/StreamThrottler.java  |   33 +
 .../org/apache/nifi/stream/io/StreamUtils.java  |  257 ++
 .../apache/nifi/stream/io/ZipOutputStream.java  |   38 +
 .../exception/BytePatternNotFoundException.java |   28 +
 .../io/util/NonThreadSafeCircularBuffer.java    |   69 +
 .../org/apache/nifi/util/BooleanHolder.java     |   25 +
 .../java/org/apache/nifi/util/FormatUtils.java  |  205 +
 .../org/apache/nifi/util/IntegerHolder.java     |   54 +
 .../java/org/apache/nifi/util/LongHolder.java   |   60 +
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  135 +
 .../java/org/apache/nifi/util/ObjectHolder.java |   39 +
 .../java/org/apache/nifi/util/RingBuffer.java   |  292 ++
 .../java/org/apache/nifi/util/StopWatch.java    |  127 +
 .../main/java/org/apache/nifi/util/Tuple.java   |   83 +
 .../concurrency/DebugDisabledTimedLock.java     |   67 +
 .../util/concurrency/DebugEnabledTimedLock.java |  136 +
 .../util/concurrency/DebuggableTimedLock.java   |   30 +
 .../apache/nifi/util/concurrency/TimedLock.java |   59 +
 .../org/apache/nifi/util/file/FileUtils.java    |  623 +++
 .../file/monitor/CompoundUpdateMonitor.java     |  115 +
 .../util/file/monitor/LastModifiedMonitor.java  |   30 +
 .../nifi/util/file/monitor/MD5SumMonitor.java   |   51 +
 .../file/monitor/SynchronousFileWatcher.java    |  123 +
 .../nifi/util/file/monitor/UpdateMonitor.java   |   25 +
 .../org/apache/nifi/util/search/Search.java     |   57 +
 .../org/apache/nifi/util/search/SearchTerm.java |  141 +
 .../util/search/ahocorasick/AhoCorasick.java    |  155 +
 .../nifi/util/search/ahocorasick/Node.java      |   72 +
 .../util/search/ahocorasick/SearchState.java    |   63 +
 .../nifi/util/timebuffer/EntityAccess.java      |   26 +
 .../nifi/util/timebuffer/LongEntityAccess.java  |   43 +
 .../nifi/util/timebuffer/TimedBuffer.java       |  114 +
 .../nifi/util/timebuffer/TimestampedLong.java   |   35 +
 .../io/TestCompressionInputOutputStreams.java   |  153 +
 .../stream/io/TestLeakyBucketThrottler.java     |  147 +
 .../nifi/util/TestNaiveSearchRingBuffer.java    |   72 +
 .../file/monitor/TestCompoundUpdateMonitor.java |   71 +
 .../monitor/TestSynchronousFileWatcher.java     |   61 +
 .../nifi/util/timebuffer/TestRingBuffer.java    |  182 +
 .../nifi/util/timebuffer/TestTimedBuffer.java   |  106 +
 .../src/test/resources/logback-test.xml         |   32 +
 nifi/commons/nifi-web-utils/pom.xml             |   56 +
 .../org/apache/nifi/web/util/ClientUtils.java   |  132 +
 .../nifi/web/util/ObjectMapperResolver.java     |   48 +
 .../java/org/apache/nifi/web/util/WebUtils.java |  198 +
 nifi/commons/pom.xml                            |   44 +
 nifi/commons/processor-utilities/pom.xml        |   45 +
 .../nifi/processor/util/FlowFileFilters.java    |   65 +
 .../nifi/processor/util/SSLProperties.java      |  226 ++
 .../nifi/processor/util/StandardValidators.java |  544 +++
 .../apache/nifi/processor/TestFormatUtils.java  |   40 +
 .../processor/util/TestStandardValidators.java  |   54 +
 nifi/commons/wali/.gitignore                    |    2 +
 nifi/commons/wali/pom.xml                       |   41 +
 .../org/wali/MinimalLockingWriteAheadLog.java   | 1008 +++++
 .../wali/src/main/java/org/wali/SerDe.java      |  128 +
 .../src/main/java/org/wali/SyncListener.java    |   62 +
 .../wali/src/main/java/org/wali/UpdateType.java |   49 +
 .../java/org/wali/WriteAheadRepository.java     |  122 +
 .../src/test/java/org/wali/DummyRecord.java     |   61 +
 .../test/java/org/wali/DummyRecordSerde.java    |  107 +
 .../wali/TestMinimalLockingWriteAheadLog.java   |  298 ++
 .../execute-script-processors/pom.xml           |   81 +
 .../nifi/processors/script/ExecuteScript.java   |  566 +++
 .../apache/nifi/scripting/ConverterScript.java  |  131 +
 .../nifi/scripting/JRubyScriptFactory.java      |   46 +
 .../nifi/scripting/JavaScriptScriptFactory.java |   56 +
 .../nifi/scripting/JythonScriptFactory.java     |   45 +
 .../nifi/scripting/OutputStreamHandler.java     |   24 +
 .../org/apache/nifi/scripting/ReaderScript.java |   79 +
 .../java/org/apache/nifi/scripting/Script.java  |  303 ++
 .../nifi/scripting/ScriptEngineFactory.java     |  117 +
 .../apache/nifi/scripting/ScriptFactory.java    |  269 ++
 .../org/apache/nifi/scripting/WriterScript.java |   67 +
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  264 ++
 .../processors/script/TestExecuteScript.java    |  939 +++++
 .../src/test/resources/alwaysFail.js            |   24 +
 .../src/test/resources/alwaysFail.py            |   19 +
 .../src/test/resources/alwaysFail.rb            |   21 +
 .../src/test/resources/ffTest.js                |   28 +
 .../src/test/resources/ffTest.py                |   22 +
 .../src/test/resources/ffTest.rb                |   30 +
 .../src/test/resources/lib/Sub.py               |   18 +
 .../src/test/resources/lib/sub.js               |   22 +
 .../src/test/resources/lib/sub.rb               |   17 +
 .../src/test/resources/loadLocal.js             |   30 +
 .../src/test/resources/loadLocal.py             |   26 +
 .../src/test/resources/loadLocal.rb             |   29 +
 .../src/test/resources/log4j.xml                |   54 +
 .../src/test/resources/optionalValidators.js    |   28 +
 .../src/test/resources/optionalValidators.py    |   22 +
 .../src/test/resources/optionalValidators.rb    |   39 +
 .../src/test/resources/paramTest.js             |   28 +
 .../src/test/resources/paramTest.py             |   26 +
 .../src/test/resources/paramTest.rb             |   31 +
 .../src/test/resources/parseXml.js              |   36 +
 .../src/test/resources/readTest.js              |   30 +
 .../src/test/resources/readTest.py              |   32 +
 .../src/test/resources/readTest.rb              |   30 +
 .../src/test/resources/readWithParams.js        |   32 +
 .../src/test/resources/readWithParams.py        |   32 +
 .../src/test/resources/readWithParams.rb        |   33 +
 .../src/test/resources/routeTest.js             |   41 +
 .../src/test/resources/routeTest.py             |   37 +
 .../src/test/resources/routeTest.rb             |   39 +
 .../src/test/resources/simpleConverter.js       |   45 +
 .../src/test/resources/simpleConverter.py       |   60 +
 .../src/test/resources/simpleConverter.rb       |   42 +
 .../src/test/resources/writeTest.js             |   26 +
 .../src/test/resources/writeTest.py             |   22 +
 .../src/test/resources/writeTest.rb             |   32 +
 .../execute-script-bundle/nar/pom.xml           |   36 +
 nifi/nar-bundles/execute-script-bundle/pom.xml  |   81 +
 .../framework/administration/.gitignore         |    1 +
 .../framework/administration/pom.xml            |  116 +
 .../nifi/admin/AuditDataSourceFactoryBean.java  |  222 ++
 .../org/apache/nifi/admin/RepositoryUtils.java  |   91 +
 .../nifi/admin/UserDataSourceFactoryBean.java   |  247 ++
 .../org/apache/nifi/admin/dao/ActionDAO.java    |   74 +
 .../org/apache/nifi/admin/dao/AuthorityDAO.java |   58 +
 .../org/apache/nifi/admin/dao/DAOFactory.java   |   29 +
 .../nifi/admin/dao/DataAccessException.java     |   39 +
 .../java/org/apache/nifi/admin/dao/UserDAO.java |  127 +
 .../nifi/admin/dao/impl/DAOFactoryImpl.java     |   51 +
 .../nifi/admin/dao/impl/StandardActionDAO.java  | 1056 +++++
 .../admin/dao/impl/StandardAuthorityDAO.java    |  172 +
 .../nifi/admin/dao/impl/StandardUserDAO.java    |  634 +++
 .../admin/service/AccountDisabledException.java |   40 +
 .../admin/service/AccountNotFoundException.java |   40 +
 .../admin/service/AccountPendingException.java  |   41 +
 .../admin/service/AdministrationException.java  |   39 +
 .../apache/nifi/admin/service/AuditService.java |   76 +
 .../apache/nifi/admin/service/UserService.java  |  161 +
 .../service/action/AbstractUserAction.java      |   97 +
 .../admin/service/action/AddActionsAction.java  |   48 +
 .../service/action/AdministrationAction.java    |   38 +
 .../service/action/AuthorizeDownloadAction.java |   54 +
 .../service/action/AuthorizeUserAction.java     |  175 +
 .../admin/service/action/CreateUserAction.java  |   53 +
 .../admin/service/action/DeleteUserAction.java  |   68 +
 .../admin/service/action/DisableUserAction.java |   76 +
 .../service/action/DisableUserGroupAction.java  |   69 +
 .../service/action/FindUserByDnAction.java      |   49 +
 .../service/action/FindUserByIdAction.java      |   49 +
 .../admin/service/action/GetActionAction.java   |   41 +
 .../admin/service/action/GetActionsAction.java  |   48 +
 .../admin/service/action/GetPreviousValues.java |   43 +
 .../service/action/GetUserGroupAction.java      |   50 +
 .../admin/service/action/GetUsersAction.java    |   39 +
 .../service/action/HasPendingUserAccounts.java  |   34 +
 .../action/InvalidateUserAccountAction.java     |   58 +
 .../InvalidateUserGroupAccountsAction.java      |   45 +
 .../service/action/PurgeActionsAction.java      |   51 +
 .../action/RequestUserAccountAction.java        |   67 +
 .../service/action/SeedUserAccountsAction.java  |  164 +
 .../admin/service/action/UngroupUserAction.java |   69 +
 .../service/action/UngroupUserGroupAction.java  |   57 +
 .../admin/service/action/UpdateUserAction.java  |  124 +
 .../UpdateUserAuthoritiesCacheAction.java       |   73 +
 .../service/action/UpdateUserCacheAction.java   |   47 +
 .../service/action/UpdateUserGroupAction.java   |  171 +
 .../service/impl/StandardAuditService.java      |  230 ++
 .../admin/service/impl/StandardUserService.java |  663 ++++
 .../admin/service/transaction/Transaction.java  |   49 +
 .../service/transaction/TransactionBuilder.java |   25 +
 .../transaction/TransactionException.java       |   40 +
 .../transaction/impl/StandardTransaction.java   |   93 +
 .../impl/StandardTransactionBuilder.java        |   57 +
 .../AuthorityProviderFactoryBean.java           |  529 +++
 ...rdAuthorityProviderConfigurationContext.java |   50 +
 ...dAuthorityProviderInitializationContext.java |   42 +
 .../java/org/apache/nifi/history/History.java   |   56 +
 .../org/apache/nifi/history/HistoryQuery.java   |   99 +
 .../org/apache/nifi/history/PreviousValue.java  |   54 +
 .../org/apache/nifi/user/AccountStatus.java     |   47 +
 .../java/org/apache/nifi/user/NiFiUser.java     |  164 +
 .../org/apache/nifi/user/NiFiUserGroup.java     |   45 +
 .../resources/nifi-administration-context.xml   |   62 +
 .../src/main/xsd/authority-providers.xsd        |   49 +
 .../service/action/AuthorizeUserActionTest.java |  433 +++
 .../service/action/CreateUserActionTest.java    |  144 +
 .../service/action/DisableUserActionTest.java   |  171 +
 .../action/InvalidateUserAccountActionTest.java |  131 +
 .../action/RequestUserAccountActionTest.java    |  127 +
 .../action/SeedUserAccountsActionTest.java      |  263 ++
 .../action/SetUserAuthoritiesActionTest.java    |  223 ++
 .../framework/client-dto/.gitignore             |    6 +
 .../framework/client-dto/pom.xml                |   46 +
 .../org/apache/nifi/web/api/dto/AboutDTO.java   |   57 +
 .../org/apache/nifi/web/api/dto/BannerDTO.java  |   57 +
 .../nifi/web/api/dto/BulletinBoardDTO.java      |   63 +
 .../apache/nifi/web/api/dto/BulletinDTO.java    |  161 +
 .../nifi/web/api/dto/BulletinQueryDTO.java      |  113 +
 .../org/apache/nifi/web/api/dto/ClusterDTO.java |   61 +
 .../apache/nifi/web/api/dto/ConnectableDTO.java |  146 +
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |  215 ++
 .../web/api/dto/ControllerConfigurationDTO.java |  158 +
 .../apache/nifi/web/api/dto/ControllerDTO.java  |  262 ++
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   94 +
 .../apache/nifi/web/api/dto/CountersDTO.java    |   61 +
 .../nifi/web/api/dto/DocumentedTypeDTO.java     |   71 +
 .../apache/nifi/web/api/dto/FlowSnippetDTO.java |  141 +
 .../org/apache/nifi/web/api/dto/FunnelDTO.java  |   29 +
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   94 +
 .../nifi/web/api/dto/NiFiComponentDTO.java      |   95 +
 .../org/apache/nifi/web/api/dto/NodeDTO.java    |  188 +
 .../apache/nifi/web/api/dto/NodeEventDTO.java   |   74 +
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |   56 +
 .../org/apache/nifi/web/api/dto/PortDTO.java    |  161 +
 .../apache/nifi/web/api/dto/PositionDTO.java    |   65 +
 .../nifi/web/api/dto/PreviousValueDTO.java      |   73 +
 .../nifi/web/api/dto/ProcessGroupDTO.java       |  219 ++
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |  486 +++
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  181 +
 .../nifi/web/api/dto/ProcessorHistoryDTO.java   |   56 +
 .../nifi/web/api/dto/PropertyHistoryDTO.java    |   43 +
 .../nifi/web/api/dto/RelationshipDTO.java       |   69 +
 .../api/dto/RemoteProcessGroupContentsDTO.java  |   56 +
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  279 ++
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |  192 +
 .../apache/nifi/web/api/dto/RevisionDTO.java    |   63 +
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |  239 ++
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  461 +++
 .../apache/nifi/web/api/dto/TemplateDTO.java    |  117 +
 .../org/apache/nifi/web/api/dto/UserDTO.java    |  177 +
 .../apache/nifi/web/api/dto/UserGroupDTO.java   |   84 +
 .../nifi/web/api/dto/action/ActionDTO.java      |  176 +
 .../nifi/web/api/dto/action/HistoryDTO.java     |   74 +
 .../web/api/dto/action/HistoryQueryDTO.java     |  144 +
 .../component/details/ComponentDetailsDTO.java  |   32 +
 .../component/details/ProcessorDetailsDTO.java  |   41 +
 .../details/RemoteProcessGroupDetailsDTO.java   |   41 +
 .../dto/action/details/ActionDetailsDTO.java    |   34 +
 .../dto/action/details/ConfigureDetailsDTO.java |   69 +
 .../dto/action/details/ConnectDetailsDTO.java   |  126 +
 .../api/dto/action/details/MoveDetailsDTO.java  |   83 +
 .../api/dto/action/details/PurgeDetailsDTO.java |   45 +
 .../web/api/dto/provenance/AttributeDTO.java    |   69 +
 .../web/api/dto/provenance/ProvenanceDTO.java   |  165 +
 .../api/dto/provenance/ProvenanceEventDTO.java  |  630 +++
 .../dto/provenance/ProvenanceOptionsDTO.java    |   43 +
 .../dto/provenance/ProvenanceRequestDTO.java    |  118 +
 .../dto/provenance/ProvenanceResultsDTO.java    |  136 +
 .../ProvenanceSearchableFieldDTO.java           |   84 +
 .../api/dto/provenance/lineage/LineageDTO.java  |  161 +
 .../provenance/lineage/LineageRequestDTO.java   |   88 +
 .../provenance/lineage/LineageResultsDTO.java   |   73 +
 .../provenance/lineage/ProvenanceLinkDTO.java   |  101 +
 .../provenance/lineage/ProvenanceNodeDTO.java   |  162 +
 .../dto/search/ComponentSearchResultDTO.java    |   85 +
 .../web/api/dto/search/NodeSearchResultDTO.java |   56 +
 .../web/api/dto/search/SearchResultsDTO.java    |  128 +
 .../dto/search/UserGroupSearchResultDTO.java    |   42 +
 .../web/api/dto/search/UserSearchResultDTO.java |   56 +
 .../dto/status/ClusterConnectionStatusDTO.java  |   89 +
 .../api/dto/status/ClusterPortStatusDTO.java    |   89 +
 .../dto/status/ClusterProcessorStatusDTO.java   |  117 +
 .../ClusterRemoteProcessGroupStatusDTO.java     |   89 +
 .../web/api/dto/status/ClusterStatusDTO.java    |   44 +
 .../api/dto/status/ClusterStatusHistoryDTO.java |   75 +
 .../web/api/dto/status/ConnectionStatusDTO.java |  198 +
 .../web/api/dto/status/ControllerStatusDTO.java |  187 +
 .../api/dto/status/NodeConnectionStatusDTO.java |   57 +
 .../web/api/dto/status/NodePortStatusDTO.java   |   57 +
 .../api/dto/status/NodeProcessorStatusDTO.java  |   57 +
 .../status/NodeRemoteProcessGroupStatusDTO.java |   57 +
 .../nifi/web/api/dto/status/NodeStatusDTO.java  |   57 +
 .../api/dto/status/NodeStatusHistoryDTO.java    |   57 +
 .../nifi/web/api/dto/status/PortStatusDTO.java  |  142 +
 .../api/dto/status/ProcessGroupStatusDTO.java   |  244 ++
 .../web/api/dto/status/ProcessorStatusDTO.java  |  203 +
 .../web/api/dto/status/RemotePortStatusDTO.java |   98 +
 .../dto/status/RemoteProcessGroupStatusDTO.java |  159 +
 .../nifi/web/api/dto/status/StatusDTO.java      |   43 +
 .../web/api/dto/status/StatusDescriptorDTO.java |  101 +
 .../web/api/dto/status/StatusHistoryDTO.java    |   92 +
 .../api/dto/status/StatusHistoryDetailDTO.java  |   56 +
 .../web/api/dto/status/StatusSnapshotDTO.java   |   58 +
 .../nifi/web/api/dto/util/DateTimeAdapter.java  |   47 +
 .../nifi/web/api/dto/util/TimeAdapter.java      |   47 +
 .../nifi/web/api/dto/util/TimestampAdapter.java |   47 +
 .../apache/nifi/web/api/entity/AboutEntity.java |   45 +
 .../nifi/web/api/entity/ActionEntity.java       |   45 +
 .../nifi/web/api/entity/AuthorityEntity.java    |   60 +
 .../nifi/web/api/entity/BannerEntity.java       |   46 +
 .../web/api/entity/BulletinBoardEntity.java     |   45 +
 .../entity/ClusterConnectionStatusEntity.java   |   45 +
 .../nifi/web/api/entity/ClusterEntity.java      |   45 +
 .../web/api/entity/ClusterPortStatusEntity.java |   45 +
 .../entity/ClusterProcessorStatusEntity.java    |   45 +
 .../ClusterRemoteProcessGroupStatusEntity.java  |   45 +
 .../api/entity/ClusterSearchResultsEntity.java  |   46 +
 .../web/api/entity/ClusterStatusEntity.java     |   45 +
 .../api/entity/ClusterStatusHistoryEntity.java  |   45 +
 .../nifi/web/api/entity/ConnectionEntity.java   |   45 +
 .../nifi/web/api/entity/ConnectionsEntity.java  |   47 +
 .../entity/ControllerConfigurationEntity.java   |   45 +
 .../nifi/web/api/entity/ControllerEntity.java   |   45 +
 .../web/api/entity/ControllerStatusEntity.java  |   45 +
 .../nifi/web/api/entity/CounterEntity.java      |   45 +
 .../nifi/web/api/entity/CountersEntity.java     |   46 +
 .../org/apache/nifi/web/api/entity/Entity.java  |   43 +
 .../nifi/web/api/entity/FlowSnippetEntity.java  |   45 +
 .../nifi/web/api/entity/FunnelEntity.java       |   45 +
 .../nifi/web/api/entity/FunnelsEntity.java      |   47 +
 .../nifi/web/api/entity/HistoryEntity.java      |   45 +
 .../nifi/web/api/entity/InputPortEntity.java    |   45 +
 .../nifi/web/api/entity/InputPortsEntity.java   |   47 +
 .../apache/nifi/web/api/entity/LabelEntity.java |   45 +
 .../nifi/web/api/entity/LabelsEntity.java       |   47 +
 .../nifi/web/api/entity/LineageEntity.java      |   45 +
 .../apache/nifi/web/api/entity/NodeEntity.java  |   45 +
 .../nifi/web/api/entity/NodeStatusEntity.java   |   45 +
 .../api/entity/NodeSystemDiagnosticsEntity.java |   45 +
 .../nifi/web/api/entity/OutputPortEntity.java   |   45 +
 .../nifi/web/api/entity/OutputPortsEntity.java  |   47 +
 .../web/api/entity/PrioritizerTypesEntity.java  |   46 +
 .../nifi/web/api/entity/ProcessGroupEntity.java |   45 +
 .../api/entity/ProcessGroupStatusEntity.java    |   45 +
 .../web/api/entity/ProcessGroupsEntity.java     |   46 +
 .../nifi/web/api/entity/ProcessorEntity.java    |   45 +
 .../web/api/entity/ProcessorHistoryEntity.java  |   45 +
 .../web/api/entity/ProcessorTypesEntity.java    |   46 +
 .../nifi/web/api/entity/ProcessorsEntity.java   |   47 +
 .../nifi/web/api/entity/ProvenanceEntity.java   |   40 +
 .../web/api/entity/ProvenanceEventEntity.java   |   45 +
 .../web/api/entity/ProvenanceOptionsEntity.java |   46 +
 .../api/entity/RemoteProcessGroupEntity.java    |   45 +
 .../entity/RemoteProcessGroupPortEntity.java    |   45 +
 .../api/entity/RemoteProcessGroupsEntity.java   |   47 +
 .../web/api/entity/SearchResultsEntity.java     |   46 +
 .../nifi/web/api/entity/SnippetEntity.java      |   45 +
 .../web/api/entity/StatusHistoryEntity.java     |   45 +
 .../web/api/entity/SystemDiagnosticsEntity.java |   45 +
 .../nifi/web/api/entity/TemplateEntity.java     |   45 +
 .../nifi/web/api/entity/TemplatesEntity.java    |   63 +
 .../apache/nifi/web/api/entity/UserEntity.java  |   45 +
 .../nifi/web/api/entity/UserGroupEntity.java    |   45 +
 .../web/api/entity/UserSearchResultsEntity.java |   61 +
 .../apache/nifi/web/api/entity/UsersEntity.java |   64 +
 .../cluster-authorization-provider/.gitignore   |    1 +
 .../cluster-authorization-provider/pom.xml      |   48 +
 .../ClusterManagerAuthorizationProvider.java    |  225 ++
 .../NodeAuthorizationProvider.java              |  389 ++
 .../protocol/message/DoesDnExistMessage.java    |   56 +
 .../protocol/message/GetAuthoritiesMessage.java |   58 +
 .../message/GetGroupForUserMessage.java         |   55 +
 .../protocol/message/ProtocolMessage.java       |   57 +
 .../message/jaxb/JaxbProtocolUtils.java         |   42 +
 .../protocol/message/jaxb/ObjectFactory.java    |   45 +
 ....apache.nifi.authorization.AuthorityProvider |   16 +
 .../framework/cluster-protocol/.gitignore       |    1 +
 .../framework/cluster-protocol/pom.xml          |   69 +
 .../protocol/ClusterManagerProtocolSender.java  |   69 +
 .../cluster/protocol/ConnectionRequest.java     |   44 +
 .../cluster/protocol/ConnectionResponse.java    |  141 +
 .../apache/nifi/cluster/protocol/Heartbeat.java |   68 +
 .../nifi/cluster/protocol/NodeBulletins.java    |   44 +
 .../nifi/cluster/protocol/NodeIdentifier.java   |  172 +
 .../cluster/protocol/NodeProtocolSender.java    |   73 +
 .../nifi/cluster/protocol/ProtocolContext.java  |   39 +
 .../cluster/protocol/ProtocolException.java     |   40 +
 .../nifi/cluster/protocol/ProtocolHandler.java  |   44 +
 .../nifi/cluster/protocol/ProtocolListener.java |   72 +
 .../protocol/ProtocolMessageMarshaller.java     |   38 +
 .../protocol/ProtocolMessageUnmarshaller.java   |   38 +
 .../nifi/cluster/protocol/StandardDataFlow.java |  105 +
 .../UnknownServiceAddressException.java         |   39 +
 .../impl/ClusterManagerProtocolSenderImpl.java  |  245 ++
 .../ClusterManagerProtocolSenderListener.java   |  118 +
 .../protocol/impl/ClusterServiceDiscovery.java  |  181 +
 .../protocol/impl/ClusterServiceLocator.java    |  229 ++
 .../impl/ClusterServicesBroadcaster.java        |  182 +
 .../protocol/impl/CopyingInputStream.java       |   77 +
 .../impl/MulticastProtocolListener.java         |  204 +
 .../protocol/impl/NodeProtocolSenderImpl.java   |  171 +
 .../impl/NodeProtocolSenderListener.java        |  115 +
 .../protocol/impl/SocketProtocolListener.java   |  205 +
 .../protocol/jaxb/JaxbProtocolContext.java      |  148 +
 .../jaxb/message/AdaptedConnectionRequest.java  |   40 +
 .../jaxb/message/AdaptedConnectionResponse.java |  109 +
 .../protocol/jaxb/message/AdaptedCounter.java   |   56 +
 .../protocol/jaxb/message/AdaptedDataFlow.java  |   64 +
 .../protocol/jaxb/message/AdaptedHeartbeat.java |   66 +
 .../jaxb/message/AdaptedNodeBulletins.java      |   50 +
 .../jaxb/message/AdaptedNodeIdentifier.java     |   76 +
 .../jaxb/message/ConnectionRequestAdapter.java  |   41 +
 .../jaxb/message/ConnectionResponseAdapter.java |   55 +
 .../protocol/jaxb/message/DataFlowAdapter.java  |   50 +
 .../protocol/jaxb/message/HeartbeatAdapter.java |   54 +
 .../jaxb/message/JaxbProtocolUtils.java         |   42 +
 .../jaxb/message/NodeBulletinsAdapter.java      |   48 +
 .../jaxb/message/NodeIdentifierAdapter.java     |   51 +
 .../protocol/jaxb/message/ObjectFactory.java    |  104 +
 .../message/ConnectionRequestMessage.java       |   46 +
 .../message/ConnectionResponseMessage.java      |   66 +
 .../ControllerStartupFailureMessage.java        |   49 +
 .../protocol/message/DisconnectMessage.java     |   55 +
 .../protocol/message/ExceptionMessage.java      |   44 +
 .../protocol/message/FlowRequestMessage.java    |   46 +
 .../protocol/message/FlowResponseMessage.java   |   44 +
 .../protocol/message/HeartbeatMessage.java      |   43 +
 .../message/MulticastProtocolMessage.java       |   66 +
 .../protocol/message/NodeBulletinsMessage.java  |   43 +
 .../cluster/protocol/message/PingMessage.java   |   55 +
 .../message/PrimaryRoleAssignmentMessage.java   |   56 +
 .../protocol/message/ProtocolMessage.java       |   61 +
 .../message/ReconnectionFailureMessage.java     |   45 +
 .../message/ReconnectionRequestMessage.java     |   94 +
 .../message/ReconnectionResponseMessage.java    |   32 +
 .../message/ServiceBroadcastMessage.java        |   64 +
 .../MulticastConfigurationFactoryBean.java      |   60 +
 .../ServerSocketConfigurationFactoryBean.java   |   65 +
 .../spring/SocketConfigurationFactoryBean.java  |   66 +
 .../resources/nifi-cluster-protocol-context.xml |  110 +
 .../ClusterManagerProtocolSenderImplTest.java   |  134 +
 .../impl/ClusterServiceDiscoveryTest.java       |  135 +
 .../impl/ClusterServiceLocatorTest.java         |  121 +
 .../impl/ClusterServicesBroadcasterTest.java    |  133 +
 .../impl/MulticastProtocolListenerTest.java     |  171 +
 .../impl/NodeProtocolSenderImplTest.java        |  203 +
 .../impl/testutils/DelayedProtocolHandler.java  |   57 +
 .../testutils/ReflexiveProtocolHandler.java     |   47 +
 .../framework/cluster-web/.gitignore            |    1 +
 .../framework/cluster-web/pom.xml               |   50 +
 .../nifi/cluster/context/ClusterContext.java    |   59 +
 .../cluster/context/ClusterContextImpl.java     |   69 +
 .../context/ClusterContextThreadLocal.java      |   47 +
 .../ClusterAwareOptimisticLockingManager.java   |   96 +
 .../framework/cluster/.gitignore                |    1 +
 .../framework-bundle/framework/cluster/pom.xml  |  132 +
 .../cluster/client/MulticastTestClient.java     |  151 +
 .../org/apache/nifi/cluster/event/Event.java    |  122 +
 .../apache/nifi/cluster/event/EventManager.java |   65 +
 .../cluster/event/impl/EventManagerImpl.java    |  143 +
 .../cluster/firewall/ClusterNodeFirewall.java   |   35 +
 .../impl/FileBasedClusterNodeFirewall.java      |  207 +
 .../nifi/cluster/flow/ClusterDataFlow.java      |   45 +
 .../apache/nifi/cluster/flow/DaoException.java  |   40 +
 .../apache/nifi/cluster/flow/DataFlowDao.java   |   62 +
 .../cluster/flow/DataFlowManagementService.java |  115 +
 .../nifi/cluster/flow/PersistedFlowState.java   |   37 +
 .../nifi/cluster/flow/StaleFlowException.java   |   42 +
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |  600 +++
 .../impl/DataFlowManagementServiceImpl.java     |  356 ++
 .../nifi/cluster/manager/ClusterManager.java    |  225 ++
 .../cluster/manager/HttpClusterManager.java     |  169 +
 .../cluster/manager/HttpRequestReplicator.java  |   99 +
 .../cluster/manager/HttpResponseMapper.java     |   42 +
 .../nifi/cluster/manager/NodeResponse.java      |  329 ++
 .../exception/BlockedByFirewallException.java   |   60 +
 .../manager/exception/ClusterException.java     |   40 +
 .../ConnectingNodeMutableRequestException.java  |   41 +
 ...DisconnectedNodeMutableRequestException.java |   41 +
 .../exception/IllegalClusterStateException.java |   41 +
 .../exception/IllegalNodeDeletionException.java |   41 +
 .../IllegalNodeDisconnectionException.java      |   42 +
 .../IllegalNodeReconnectionException.java       |   41 +
 .../IneligiblePrimaryNodeException.java         |   41 +
 .../exception/MutableRequestException.java      |   42 +
 .../exception/NoConnectedNodesException.java    |   41 +
 .../exception/NoResponseFromNodesException.java |   42 +
 .../exception/NodeDisconnectionException.java   |   41 +
 .../exception/NodeReconnectionException.java    |   40 +
 .../PrimaryRoleAssignmentException.java         |   41 +
 .../SafeModeMutableRequestException.java        |   41 +
 .../manager/exception/UnknownNodeException.java |   41 +
 .../exception/UriConstructionException.java     |   42 +
 .../manager/impl/ClusteredEventAccess.java      |  135 +
 .../manager/impl/ClusteredReportingContext.java |  165 +
 .../manager/impl/HttpRequestReplicatorImpl.java |  531 +++
 .../manager/impl/HttpResponseMapperImpl.java    |   85 +
 .../cluster/manager/impl/WebClusterManager.java | 3620 ++++++++++++++++++
 .../java/org/apache/nifi/cluster/node/Node.java |  252 ++
 ...anagerProtocolServiceLocatorFactoryBean.java |  116 +
 ...FileBasedClusterNodeFirewallFactoryBean.java |   58 +
 .../spring/WebClusterManagerFactoryBean.java    |  139 +
 .../reporting/ClusteredReportingTaskNode.java   |   49 +
 .../resources/nifi-cluster-manager-context.xml  |  124 +
 .../event/impl/EventManagerImplTest.java        |  119 +
 .../impl/FileBasedClusterNodeFirewallTest.java  |   98 +
 .../impl/DataFlowManagementServiceImplTest.java |  343 ++
 .../impl/HttpRequestReplicatorImplTest.java     |  368 ++
 .../impl/HttpResponseMapperImplTest.java        |  126 +
 .../manager/impl/TestWebClusterManager.java     |   54 +
 .../cluster/manager/testutils/HttpRequest.java  |  239 ++
 .../cluster/manager/testutils/HttpResponse.java |   93 +
 .../manager/testutils/HttpResponseAction.java   |   60 +
 .../cluster/manager/testutils/HttpServer.java   |  240 ++
 .../ClusterManagerProtocolSenderImplTest.java   |  133 +
 .../impl/ClusterServiceLocatorTest.java         |  119 +
 .../impl/ClusterServicesBroadcasterTest.java    |  131 +
 .../impl/MulticastProtocolListenerTest.java     |  171 +
 .../impl/NodeProtocolSenderImplTest.java        |  201 +
 .../impl/SocketProtocolListenerTest.java        |  132 +
 .../testutils/DelayedProtocolHandler.java       |   57 +
 .../testutils/ReflexiveProtocolHandler.java     |   47 +
 .../cluster/src/test/resources/logback-test.xml |   48 +
 .../apache/nifi/cluster/firewall/impl/empty.txt |    0
 .../apache/nifi/cluster/firewall/impl/ips.txt   |   12 +
 .../framework/core-api/.gitignore               |    1 +
 .../framework-bundle/framework/core-api/pom.xml |   56 +
 .../nifi/cluster/AdaptedNodeInformation.java    |   66 +
 .../nifi/cluster/ClusterNodeInformation.java    |   67 +
 .../org/apache/nifi/cluster/NodeInformant.java  |   22 +
 .../apache/nifi/cluster/NodeInformation.java    |   98 +
 .../nifi/cluster/NodeInformationAdapter.java    |   39 +
 .../apache/nifi/cluster/protocol/DataFlow.java  |   41 +
 .../apache/nifi/connectable/Connectable.java    |  291 ++
 .../nifi/connectable/ConnectableType.java       |   44 +
 .../org/apache/nifi/connectable/Connection.java |   78 +
 .../org/apache/nifi/connectable/Funnel.java     |   24 +
 .../java/org/apache/nifi/connectable/Port.java  |   31 +
 .../org/apache/nifi/connectable/Position.java   |   36 +
 .../java/org/apache/nifi/connectable/Size.java  |   36 +
 .../controller/AbstractConfiguredComponent.java |  280 ++
 .../apache/nifi/controller/AbstractPort.java    |  636 +++
 .../apache/nifi/controller/Availability.java    |   24 +
 .../nifi/controller/ConfiguredComponent.java    |   63 +
 .../nifi/controller/ContentAvailability.java    |   65 +
 .../org/apache/nifi/controller/Counter.java     |   32 +
 .../nifi/controller/EventBasedWorker.java       |   32 +
 .../org/apache/nifi/controller/Heartbeater.java |   22 +
 .../nifi/controller/ProcessScheduler.java       |  146 +
 .../apache/nifi/controller/ProcessorNode.java   |   80 +
 .../nifi/controller/ReportingTaskNode.java      |   56 +
 .../nifi/controller/StandardFlowFileQueue.java  | 1096 ++++++
 .../apache/nifi/controller/StandardFunnel.java  |  541 +++
 .../controller/ValidationContextFactory.java    |   27 +
 .../org/apache/nifi/controller/WorkerQueue.java |   36 +
 .../exception/CommunicationsException.java      |   40 +
 ...ControllerServiceAlreadyExistsException.java |   30 +
 .../ControllerServiceNotFoundException.java     |   51 +
 .../ProcessorInstantiationException.java        |   27 +
 .../exception/ProcessorLifeCycleException.java  |   30 +
 .../org/apache/nifi/controller/label/Label.java |   48 +
 .../ReportingTaskInstantiationException.java    |   31 +
 .../repository/ContentNotFoundException.java    |   48 +
 .../repository/CounterRepository.java           |   34 +
 .../controller/repository/FlowFileEvent.java    |   54 +
 .../repository/FlowFileEventRepository.java     |   50 +
 .../repository/RepositoryStatusReport.java      |   28 +
 .../service/ControllerServiceNode.java          |   40 +
 .../service/ControllerServiceProvider.java      |   47 +
 .../service/ControllerServiceReference.java     |   50 +
 .../org/apache/nifi/events/BulletinFactory.java |   52 +
 .../nifi/events/BulletinProcessingStrategy.java |   27 +
 .../apache/nifi/events/ComponentBulletin.java   |   30 +
 .../org/apache/nifi/events/SystemBulletin.java  |   30 +
 .../org/apache/nifi/groups/ProcessGroup.java    |  723 ++++
 .../apache/nifi/groups/ProcessGroupCounts.java  |   66 +
 .../apache/nifi/groups/RemoteProcessGroup.java  |  255 ++
 .../RemoteProcessGroupPortDescriptor.java       |   92 +
 .../org/apache/nifi/logging/LogMessage.java     |   74 +
 .../org/apache/nifi/logging/LogObserver.java    |   22 +
 .../org/apache/nifi/logging/LogRepository.java  |   67 +
 .../nifi/logging/LogRepositoryFactory.java      |   61 +
 .../java/org/apache/nifi/nar/NarCloseable.java  |   44 +
 .../nifi/nar/NarThreadContextClassLoader.java   |  188 +
 .../main/java/org/apache/nifi/remote/Peer.java  |  107 +
 .../java/org/apache/nifi/remote/PeerStatus.java |   72 +
 .../nifi/remote/PortAuthorizationResult.java    |   25 +
 .../nifi/remote/RemoteAuthorizationState.java   |   27 +
 .../org/apache/nifi/remote/RemoteGroupPort.java |   35 +
 .../org/apache/nifi/remote/RootGroupPort.java   |   78 +
 .../apache/nifi/remote/TransferDirection.java   |   23 +
 .../nifi/remote/VersionedRemoteResource.java    |   24 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |   79 +
 .../remote/exception/BadRequestException.java   |   30 +
 .../remote/exception/HandshakeException.java    |   30 +
 .../exception/NotAuthorizedException.java       |   26 +
 .../exception/PortNotRunningException.java      |   26 +
 .../remote/exception/ProtocolException.java     |   34 +
 .../exception/RequestExpiredException.java      |   26 +
 .../remote/exception/UnknownPortException.java  |   26 +
 .../nifi/remote/protocol/ClientProtocol.java    |   78 +
 .../remote/protocol/CommunicationsInput.java    |   27 +
 .../remote/protocol/CommunicationsOutput.java   |   27 +
 .../remote/protocol/CommunicationsSession.java  |   64 +
 .../nifi/remote/protocol/RequestType.java       |   43 +
 .../nifi/remote/protocol/ServerProtocol.java    |  143 +
 .../framework-bundle/framework/core/.gitignore  |    1 +
 .../framework-bundle/framework/core/pom.xml     |  121 +
 .../apache/nifi/cluster/BulletinsPayload.java   |   95 +
 .../nifi/cluster/ConnectionException.java       |   42 +
 .../nifi/cluster/DisconnectionException.java    |   42 +
 .../apache/nifi/cluster/HeartbeatPayload.java   |  170 +
 .../org/apache/nifi/connectable/LocalPort.java  |  172 +
 .../nifi/connectable/StandardConnection.java    |  336 ++
 .../nifi/controller/EventDrivenWorkerQueue.java |  329 ++
 .../nifi/controller/FileSystemSwapManager.java  |  768 ++++
 .../apache/nifi/controller/FlowController.java  | 3579 +++++++++++++++++
 .../nifi/controller/FlowFromDOMFactory.java     |  418 ++
 .../controller/FlowSerializationException.java  |   48 +
 .../apache/nifi/controller/FlowSerializer.java  |   42 +
 .../FlowSynchronizationException.java           |   47 +
 .../nifi/controller/FlowSynchronizer.java       |   53 +
 .../nifi/controller/FlowUnmarshaller.java       |   78 +
 .../apache/nifi/controller/SnippetManager.java  |   96 +
 .../apache/nifi/controller/StandardCounter.java |  108 +
 .../nifi/controller/StandardFlowSerializer.java |  404 ++
 .../nifi/controller/StandardFlowService.java    |  875 +++++
 .../controller/StandardFlowSynchronizer.java    | 1026 +++++
 .../nifi/controller/StandardProcessorNode.java  | 1243 ++++++
 .../apache/nifi/controller/StandardSnippet.java |  186 +
 .../org/apache/nifi/controller/Template.java    |   37 +
 .../apache/nifi/controller/TemplateManager.java |  507 +++
 .../controller/UninheritableFlowException.java  |   48 +
 .../exception/FlowFileConsumptionException.java |   38 +
 .../ReportingTaskInstantiationException.java    |   31 +
 .../exception/ValidationException.java          |   47 +
 .../nifi/controller/label/StandardLabel.java    |  110 +
 .../reporting/AbstractReportingTaskNode.java    |  111 +
 .../reporting/StandardReportingContext.java     |  132 +
 .../StandardReportingInitializationContext.java |   96 +
 .../reporting/StandardReportingTaskNode.java    |   40 +
 .../repository/BatchingSessionFactory.java      |  242 ++
 .../repository/ConnectionSwapInfo.java          |   58 +
 .../repository/FileSystemRepository.java        | 1351 +++++++
 .../controller/repository/ProcessContext.java   |  291 ++
 .../repository/ProvenanceEventEnricher.java     |   34 +
 .../repository/RepositoryPurgeException.java    |   59 +
 .../repository/RingBufferEventRepository.java   |  312 ++
 .../repository/StandardCounterRepository.java   |  109 +
 .../repository/StandardFlowFileEvent.java       |  237 ++
 .../repository/StandardFlowFileRecord.java      |  312 ++
 .../repository/StandardProcessSession.java      | 2684 +++++++++++++
 .../StandardProcessSessionFactory.java          |   33 +
 .../repository/StandardProvenanceReporter.java  |  431 +++
 .../repository/StandardRepositoryRecord.java    |  196 +
 .../StandardRepositoryStatusReport.java         |   88 +
 .../repository/VolatileContentRepository.java   |  647 ++++
 .../repository/VolatileFlowFileRepository.java  |  109 +
 .../WriteAheadFlowFileRepository.java           |  864 +++++
 .../repository/claim/ContentDirection.java      |   35 +
 .../repository/claim/StandardContentClaim.java  |  154 +
 .../claim/StandardContentClaimManager.java      |  155 +
 .../repository/io/ArrayManagedOutputStream.java |  245 ++
 .../repository/io/ByteCountingInputStream.java  |  100 +
 .../repository/io/ByteCountingOutputStream.java |   64 +
 .../io/DisableOnCloseOutputStream.java          |   73 +
 .../io/FlowFileAccessInputStream.java           |  168 +
 .../io/FlowFileAccessOutputStream.java          |   95 +
 .../repository/io/LimitedInputStream.java       |  107 +
 .../controller/repository/io/LongHolder.java    |   48 +
 .../controller/repository/io/MemoryManager.java |   55 +
 .../repository/io/SyncOnCloseOutputStream.java  |   58 +
 .../scheduling/ConnectableProcessContext.java   |  194 +
 .../scheduling/EventDrivenSchedulingAgent.java  |  346 ++
 .../scheduling/ProcessContextFactory.java       |   51 +
 .../scheduling/QuartzSchedulingAgent.java       |  220 ++
 .../controller/scheduling/ScheduleState.java    |   90 +
 .../controller/scheduling/SchedulingAgent.java  |   45 +
 .../scheduling/StandardProcessScheduler.java    |  569 +++
 .../scheduling/TimerDrivenSchedulingAgent.java  |  139 +
 .../service/ControllerServiceLoader.java        |  156 +
 .../service/StandardConfigurationContext.java   |   64 +
 ...dControllerServiceInitializationContext.java |   64 +
 .../service/StandardControllerServiceNode.java  |  125 +
 .../StandardControllerServiceProvider.java      |  196 +
 .../StandardControllerServiceReference.java     |  100 +
 .../history/StandardMetricDescriptor.java       |  128 +
 .../status/history/StandardStatusHistory.java   |   54 +
 .../status/history/StandardStatusSnapshot.java  |   70 +
 .../status/history/StatusHistoryUtil.java       |  107 +
 .../VolatileComponentStatusRepository.java      |  649 ++++
 .../tasks/ContinuallyRunConnectableTask.java    |   97 +
 .../tasks/ContinuallyRunProcessorTask.java      |  185 +
 .../nifi/controller/tasks/ExpireFlowFiles.java  |  114 +
 .../controller/tasks/ReportingTaskWrapper.java  |   63 +
 .../nifi/diagnostics/DiagnosticUtils.java       |   28 +
 .../nifi/diagnostics/GarbageCollection.java     |   60 +
 .../apache/nifi/diagnostics/StorageUsage.java   |   72 +
 .../nifi/diagnostics/SystemDiagnostics.java     |  212 +
 .../diagnostics/SystemDiagnosticsFactory.java   |  133 +
 .../nifi/encrypt/EncryptionException.java       |   43 +
 .../apache/nifi/encrypt/StringEncryptor.java    |  152 +
 .../java/org/apache/nifi/engine/FlowEngine.java |  135 +
 .../events/NodeBulletinProcessingStrategy.java  |   66 +
 .../nifi/events/VolatileBulletinRepository.java |  240 ++
 .../events/network/CommunicationsFailure.java   |   36 +
 .../nifi/events/network/NetworkTransfer.java    |   48 +
 .../nifi/fingerprint/FingerprintException.java  |   41 +
 .../nifi/fingerprint/FingerprintFactory.java    |  985 +++++
 .../nifi/groups/StandardProcessGroup.java       | 2019 ++++++++++
 .../org/apache/nifi/jaxb/AdaptedBulletin.java   |  100 +
 .../org/apache/nifi/jaxb/AdaptedCounter.java    |   64 +
 .../org/apache/nifi/jaxb/BulletinAdapter.java   |   59 +
 .../org/apache/nifi/jaxb/CounterAdapter.java    |   51 +
 .../org/apache/nifi/lifecycle/LifeCycle.java    |   54 +
 .../nifi/lifecycle/LifeCycleException.java      |   48 +
 .../nifi/lifecycle/LifeCycleStartException.java |   49 +
 .../nifi/lifecycle/LifeCycleStopException.java  |   48 +
 .../nifi/logging/ProcessorLogObserver.java      |   48 +
 .../repository/StandardLogRepository.java       |  173 +
 .../nifi/persistence/FlowConfigurationDAO.java  |  135 +
 .../StandardSnippetDeserializer.java            |   42 +
 .../persistence/StandardSnippetSerializer.java  |   47 +
 .../StandardXMLFlowConfigurationDAO.java        |  345 ++
 .../nifi/persistence/TemplateDeserializer.java  |   42 +
 .../nifi/persistence/TemplateSerializer.java    |   47 +
 .../nifi/processor/SimpleProcessLogger.java     |  307 ++
 .../nifi/processor/StandardProcessContext.java  |  173 +
 .../StandardProcessorInitializationContext.java |   49 +
 .../nifi/processor/StandardPropertyValue.java   |  163 +
 .../processor/StandardSchedulingContext.java    |  113 +
 .../processor/StandardValidationContext.java    |   97 +
 .../StandardValidationContextFactory.java       |   39 +
 .../nifi/remote/StandardRemoteProcessGroup.java | 1621 ++++++++
 ...tandardRemoteProcessGroupPortDescriptor.java |  148 +
 .../org/apache/nifi/services/FlowService.java   |  139 +
 .../nifi/spring/FlowControllerFactoryBean.java  |  103 +
 .../spring/RingBufferEventRepositoryBean.java   |   45 +
 .../spring/StandardFlowServiceFactoryBean.java  |   98 +
 .../apache/nifi/util/ComponentStatusReport.java |  139 +
 .../java/org/apache/nifi/util/Connectables.java |   56 +
 .../java/org/apache/nifi/util/DomUtils.java     |   79 +
 .../org/apache/nifi/util/ReflectionUtils.java   |  157 +
 .../java/org/apache/nifi/util/SnippetUtils.java |  241 ++
 .../ControllerServiceConfiguration.xsd          |   61 +
 .../src/main/resources/FlowConfiguration.xsd    |  335 ++
 .../resources/ReportingTaskConfiguration.xsd    |   87 +
 .../core/src/main/resources/nifi-context.xml    |   49 +
 .../nifi/cluster/HeartbeatPayloadTest.java      |  121 +
 .../controller/StandardFlowServiceTest.java     |  300 ++
 .../controller/TestFileSystemSwapManager.java   |  104 +
 .../repository/TestFileSystemRepository.java    |  352 ++
 .../TestRingBufferEventRepository.java          |  138 +
 .../repository/TestStandardProcessSession.java  | 1177 ++++++
 .../TestStandardProvenanceReporter.java         |   65 +
 .../TestVolatileContentRepository.java          |  183 +
 .../TestWriteAheadFlowFileRepository.java       |  135 +
 .../repository/io/TestLimitedOutputStream.java  |   76 +
 .../controller/repository/util/DiskUtils.java   |   77 +
 .../fingerprint/FingerprintFactoryTest.java     |   60 +
 .../processor/TestStandardPropertyValue.java    |  167 +
 .../processors/DataGeneratorTestProcessor.java  |   85 +
 .../StubAttributeLoggerProcessor.java           |  111 +
 .../StubTerminationFileProcessor.java           |   40 +
 .../core/src/test/resources/conf/0bytes.xml     |    0
 .../test/resources/conf/all-flow-corrupt.xml    |  201 +
 .../resources/conf/all-flow-inheritable.xml     |  196 +
 .../resources/conf/all-flow-uninheritable.xml   |  202 +
 .../core/src/test/resources/conf/all-flow.xml   |  198 +
 .../src/test/resources/conf/nifi.properties     |  127 +
 .../test/resources/conf/only-termination.xml    |   37 +
 .../src/test/resources/conf/remote-flow.xml     |  145 +
 .../src/test/resources/conf/standard-flow.xml   |  196 +
 .../core/src/test/resources/conf/taskConfig.xml |   17 +
 .../test/resources/conf/termination-only.xml    |    0
 .../framework/core/src/test/resources/hello.txt |    1 +
 .../core/src/test/resources/logback-test.xml    |   36 +
 .../test/resources/nifi-with-remote.properties  |  127 +
 .../core/src/test/resources/nifi.properties     |  127 +
 .../test/resources/nifi/fingerprint/flow1a.xml  |  160 +
 .../test/resources/nifi/fingerprint/flow1b.xml  |  160 +
 .../test/resources/nifi/fingerprint/flow2.xml   |  160 +
 .../core/src/test/resources/old-swap-file.swap  |  Bin 0 -> 1730054 bytes
 .../file-authorization-provider/pom.xml         |   81 +
 .../FileAuthorizationProvider.java              |  584 +++
 ....apache.nifi.authorization.AuthorityProvider |   15 +
 .../src/main/xsd/users.xsd                      |   64 +
 .../FileAuthorizationProviderTest.java          |  127 +
 .../framework-bundle/framework/nar/.gitignore   |    1 +
 .../framework-bundle/framework/nar/pom.xml      |   36 +
 .../org/apache/nifi/nar/ExtensionManager.java   |  188 +
 .../org/apache/nifi/nar/ExtensionMapping.java   |   73 +
 .../org/apache/nifi/nar/NarClassLoader.java     |  227 ++
 .../org/apache/nifi/nar/NarClassLoaders.java    |  301 ++
 .../java/org/apache/nifi/nar/NarUnpacker.java   |  351 ++
 .../java/org/apache/nifi/util/FileUtils.java    |  187 +
 .../framework-bundle/framework/pom.xml          |   51 +
 .../framework/resources/pom.xml                 |   52 +
 .../src/main/assembly/dependencies.xml          |   36 +
 .../src/main/resources/bin/dump-nifi.bat        |   33 +
 .../src/main/resources/bin/nifi-status.bat      |   33 +
 .../resources/src/main/resources/bin/nifi.sh    |  181 +
 .../src/main/resources/bin/run-nifi.bat         |   33 +
 .../src/main/resources/bin/start-nifi.bat       |   33 +
 .../src/main/resources/bin/stop-nifi.bat        |   33 +
 .../main/resources/conf/authority-providers.xml |   43 +
 .../main/resources/conf/authorized-users.xml    |   57 +
 .../src/main/resources/conf/bootstrap.conf      |   57 +
 .../main/resources/conf/controller-services.xml |   18 +
 .../src/main/resources/conf/logback.xml         |  105 +
 .../src/main/resources/conf/nifi.properties     |  159 +
 .../src/main/resources/conf/reporting-tasks.xml |   49 +
 .../framework/runtime/.gitignore                |    2 +
 .../framework-bundle/framework/runtime/pom.xml  |   42 +
 .../java/org/apache/nifi/BootstrapListener.java |  378 ++
 .../src/main/java/org/apache/nifi/NiFi.java     |  223 ++
 .../main/java/org/apache/nifi/NiFiServer.java   |   31 +
 .../apache/nifi/util/LimitingInputStream.java   |  107 +
 .../framework/security/.gitignore               |    1 +
 .../framework-bundle/framework/security/pom.xml |   42 +
 .../util/SslContextCreationException.java       |   40 +
 .../security/util/SslContextFactory.java        |  119 +
 .../framework/security/util/SslException.java   |   40 +
 .../security/util/SslServerSocketFactory.java   |   81 +
 ...SslServerSocketFactoryCreationException.java |   40 +
 .../security/util/SslSocketFactory.java         |   91 +
 .../util/SslSocketFactoryCreationException.java |   40 +
 .../security/util/SslContextFactoryTest.java    |   69 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../src/test/resources/log4j.properties         |   21 +
 .../framework/site-to-site/.gitignore           |    1 +
 .../framework/site-to-site/pom.xml              |   74 +
 .../util/RemoteProcessGroupUtils.java           |  216 ++
 .../remote/AbstractCommunicationsSession.java   |   54 +
 .../nifi/remote/RemoteResourceFactory.java      |  202 +
 .../nifi/remote/RemoteResourceManager.java      |  227 ++
 .../apache/nifi/remote/RemoteSiteListener.java  |   33 +
 .../nifi/remote/SocketRemoteSiteListener.java   |  340 ++
 .../nifi/remote/StandardRemoteGroupPort.java    |  764 ++++
 .../nifi/remote/StandardRootGroupPort.java      |  629 +++
 .../remote/codec/StandardFlowFileCodec.java     |  169 +
 .../exception/UnsupportedCodecException.java    |   31 +
 .../SocketChannelCommunicationsSession.java     |   90 +
 .../remote/io/socket/SocketChannelInput.java    |   66 +
 .../remote/io/socket/SocketChannelOutput.java   |   58 +
 .../SSLSocketChannelCommunicationsSession.java  |   93 +
 .../io/socket/ssl/SSLSocketChannelInput.java    |   50 +
 .../io/socket/ssl/SSLSocketChannelOutput.java   |   44 +
 .../socket/ClusterManagerServerProtocol.java    |  208 +
 .../protocol/socket/HandshakeProperty.java      |   23 +
 .../nifi/remote/protocol/socket/Response.java   |   51 +
 .../remote/protocol/socket/ResponseCode.java    |  152 +
 .../protocol/socket/SocketClientProtocol.java   |  510 +++
 .../socket/SocketFlowFileServerProtocol.java    |  581 +++
 .../org.apache.nifi.remote.codec.FlowFileCodec  |   15 +
 ...g.apache.nifi.remote.protocol.ClientProtocol |   15 +
 ...g.apache.nifi.remote.protocol.ServerProtocol |   16 +
 .../remote/TestStandardRemoteGroupPort.java     |   97 +
 .../remote/TestStandardSiteToSiteProtocol.java  |  116 +
 .../io/socket/TestSocketChannelStreams.java     |  231 ++
 .../io/socket/ssl/TestSSLSocketChannel.java     |  381 ++
 .../test/resources/dummy-certs/localhost-ks.jks |  Bin 0 -> 1298 bytes
 .../test/resources/dummy-certs/localhost-ts.jks |  Bin 0 -> 891 bytes
 .../src/test/resources/logback-test.xml         |   30 +
 .../src/test/resources/nifi.properties          |  111 +
 .../framework/user-actions/.gitignore           |    1 +
 .../framework/user-actions/pom.xml              |   27 +
 .../java/org/apache/nifi/action/Action.java     |  121 +
 .../java/org/apache/nifi/action/Component.java  |   31 +
 .../java/org/apache/nifi/action/Operation.java  |   36 +
 .../component/details/ComponentDetails.java     |   26 +
 .../component/details/ProcessorDetails.java     |   34 +
 .../details/RemoteProcessGroupDetails.java      |   34 +
 .../nifi/action/details/ActionDetails.java      |   26 +
 .../nifi/action/details/ConfigureDetails.java   |   52 +
 .../nifi/action/details/ConnectDetails.java     |   90 +
 .../apache/nifi/action/details/MoveDetails.java |   61 +
 .../nifi/action/details/PurgeDetails.java       |   41 +
 .../web/custom-ui-utilities/.gitignore          |    1 +
 .../framework/web/custom-ui-utilities/pom.xml   |   36 +
 .../web/HttpServletRequestContextConfig.java    |  118 +
 .../framework/web/nifi-jetty/.gitignore         |    1 +
 .../framework/web/nifi-jetty/pom.xml            |  166 +
 .../src/main/assembly/distribution.xml          |   42 +
 .../org/apache/nifi/web/server/JettyServer.java |  686 ++++
 .../server/ServerConfigurationException.java    |   36 +
 .../org/apache/nifi/web/webdefault.xml          |  556 +++
 .../framework/web/nifi-web-api/.gitignore       |    1 +
 .../framework/web/nifi-web-api/pom.xml          |  305 ++
 .../src/main/enunciate/enunciate.xml            |   36 +
 .../src/main/enunciate/images/home.png          |  Bin 0 -> 144 bytes
 .../src/main/enunciate/override.css             |  178 +
 .../org/apache/nifi/aop/MethodProfiler.java     |   45 +
 .../apache/nifi/audit/ControllerAuditor.java    |  257 ++
 .../org/apache/nifi/audit/FunnelAuditor.java    |  135 +
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  119 +
 .../java/org/apache/nifi/audit/PortAuditor.java |  334 ++
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  263 ++
 .../org/apache/nifi/audit/ProcessorAuditor.java |  408 ++
 .../apache/nifi/audit/RelationshipAuditor.java  |  412 ++
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  376 ++
 .../org/apache/nifi/audit/SnippetAuditor.java   |  519 +++
 .../nifi/web/ClusterRequestException.java       |   38 +
 .../IllegalClusterResourceRequestException.java |   33 +
 .../org/apache/nifi/web/NiFiCoreException.java  |   32 +
 .../org/apache/nifi/web/NiFiServiceFacade.java  | 1263 ++++++
 .../apache/nifi/web/NiFiServiceFacadeLock.java  |  177 +
 .../nifi/web/StandardNiFiServiceFacade.java     | 2877 ++++++++++++++
 .../apache/nifi/web/StandardNiFiWebContext.java |  336 ++
 .../nifi/web/api/ApplicationResource.java       |  399 ++
 .../nifi/web/api/BulletinBoardResource.java     |  116 +
 .../apache/nifi/web/api/ClusterResource.java    |  724 ++++
 .../apache/nifi/web/api/ConnectionResource.java |  771 ++++
 .../apache/nifi/web/api/ControllerResource.java |  803 ++++
 .../org/apache/nifi/web/api/FunnelResource.java |  493 +++
 .../apache/nifi/web/api/HistoryResource.java    |  271 ++
 .../apache/nifi/web/api/InputPortResource.java  |  531 +++
 .../org/apache/nifi/web/api/LabelResource.java  |  547 +++
 .../org/apache/nifi/web/api/NodeResource.java   |  302 ++
 .../apache/nifi/web/api/OutputPortResource.java |  532 +++
 .../nifi/web/api/ProcessGroupResource.java      | 1102 ++++++
 .../apache/nifi/web/api/ProcessorResource.java  |  693 ++++
 .../apache/nifi/web/api/ProvenanceResource.java | 1068 ++++++
 .../web/api/RemoteProcessGroupResource.java     |  912 +++++
 .../apache/nifi/web/api/SnippetResource.java    |  601 +++
 .../nifi/web/api/SystemDiagnosticsResource.java |   81 +
 .../apache/nifi/web/api/TemplateResource.java   |  400 ++
 .../apache/nifi/web/api/UserGroupResource.java  |  383 ++
 .../org/apache/nifi/web/api/UserResource.java   |  469 +++
 .../api/config/AccessDeniedExceptionMapper.java |   54 +
 .../config/AccountNotFoundExceptionMapper.java  |   47 +
 .../config/AdministrationExceptionMapper.java   |   48 +
 ...ationCredentialsNotFoundExceptionMapper.java |   48 +
 .../web/api/config/ClusterExceptionMapper.java  |   47 +
 ...ectingNodeMutableRequestExceptionMapper.java |   49 +
 ...nectedNodeMutableRequestExceptionMapper.java |   49 +
 .../config/IllegalArgumentExceptionMapper.java  |   46 +
 ...alClusterResourceRequestExceptionMapper.java |   47 +
 .../IllegalClusterStateExceptionMapper.java     |   47 +
 .../IllegalNodeDeletionExceptionMapper.java     |   46 +
 ...IllegalNodeDisconnectionExceptionMapper.java |   46 +
 .../IllegalNodeReconnectionExceptionMapper.java |   46 +
 .../api/config/IllegalStateExceptionMapper.java |   46 +
 .../IneligiblePrimaryNodeExceptionMapper.java   |   47 +
 .../config/InvalidRevisionExceptionMapper.java  |   47 +
 .../api/config/JsonMappingExceptionMapper.java  |   46 +
 .../api/config/JsonParseExceptionMapper.java    |   46 +
 .../config/MutableRequestExceptionMapper.java   |   48 +
 .../web/api/config/NiFiCoreExceptionMapper.java |   47 +
 .../config/NoConnectedNodesExceptionMapper.java |   47 +
 .../NoResponseFromNodesExceptionMapper.java     |   49 +
 .../NodeDisconnectionExceptionMapper.java       |   47 +
 .../config/NodeReconnectionExceptionMapper.java |   47 +
 .../PrimaryRoleAssignmentExceptionMapper.java   |   47 +
 .../config/ResourceNotFoundExceptionMapper.java |   48 +
 .../SafeModeMutableRequestExceptionMapper.java  |   48 +
 .../nifi/web/api/config/ThrowableMapper.java    |   46 +
 .../api/config/UnknownNodeExceptionMapper.java  |   47 +
 .../api/config/ValidationExceptionMapper.java   |   49 +
 .../config/WebApplicationExceptionMapper.java   |   63 +
 .../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 +++++++++++
 .../org/apache/nifi/web/api/package-info.java   |   58 +
 .../request/BulletinBoardPatternParameter.java  |   47 +
 .../nifi/web/api/request/ClientIdParameter.java |   45 +
 .../api/request/ConnectableTypeParameter.java   |   41 +
 .../nifi/web/api/request/DateTimeParameter.java |   63 +
 .../nifi/web/api/request/DoubleParameter.java   |   40 +
 .../nifi/web/api/request/IntegerParameter.java  |   40 +
 .../nifi/web/api/request/LongParameter.java     |   40 +
 .../ApplicationStartupContextListener.java      |  157 +
 .../nifi/web/controller/ControllerFacade.java   | 1392 +++++++
 .../web/controller/StandardSearchContext.java   |   64 +
 .../org/apache/nifi/web/dao/ConnectionDAO.java  |  112 +
 .../java/org/apache/nifi/web/dao/FunnelDAO.java |   88 +
 .../java/org/apache/nifi/web/dao/LabelDAO.java  |   79 +
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   96 +
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |   89 +
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   97 +
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |  135 +
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   92 +
 .../org/apache/nifi/web/dao/TemplateDAO.java    |   77 +
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |   72 +
 .../web/dao/impl/StandardConnectionDAO.java     |  519 +++
 .../nifi/web/dao/impl/StandardFunnelDAO.java    |  180 +
 .../nifi/web/dao/impl/StandardInputPortDAO.java |  309 ++
 .../nifi/web/dao/impl/StandardLabelDAO.java     |  184 +
 .../web/dao/impl/StandardOutputPortDAO.java     |  309 ++
 .../web/dao/impl/StandardProcessGroupDAO.java   |  171 +
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  503 +++
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  402 ++
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |  344 ++
 .../nifi/web/dao/impl/StandardTemplateDAO.java  |  141 +
 .../nifi/web/filter/NodeRequestFilter.java      |  130 +
 .../apache/nifi/web/filter/RequestLogger.java   |   78 +
 .../nifi/web/filter/ThreadLocalFilter.java      |   54 +
 .../org/apache/nifi/web/filter/TimerFilter.java |   70 +
 .../nifi/web/util/ClientResponseUtils.java      |   48 +
 .../nifi/web/util/DownloadableContent.java      |   47 +
 .../org/apache/nifi/web/util/SnippetUtils.java  |  426 +++
 .../src/main/resources/nifi-web-api-context.xml |  303 ++
 .../src/main/webapp/WEB-INF/web.xml             |  105 +
 .../apache/nifi/integration/NiFiWebApiTest.java |  274 ++
 .../accesscontrol/AdminAccessControlTest.java   | 1090 ++++++
 .../accesscontrol/DfmAccessControlTest.java     | 1410 +++++++
 .../ReadOnlyAccessControlTest.java              | 1064 +++++
 .../util/NiFiTestAuthorizationProvider.java     |  179 +
 .../nifi/integration/util/NiFiTestServer.java   |  235 ++
 .../nifi/integration/util/NiFiTestUser.java     |  257 ++
 .../integration/util/SourceTestProcessor.java   |   49 +
 .../util/TerminationTestProcessor.java          |   44 +
 .../nifi/integration/util/TestPrioritizer.java  |   31 +
 ....apache.nifi.authorization.AuthorityProvider |   15 +
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   15 +
 .../org.apache.nifi.processor.Processor         |   16 +
 .../access-control/authority-providers.xml      |   24 +
 .../access-control/controller-services.xml      |   18 +
 .../resources/access-control/localhost-ks.jks   |  Bin 0 -> 2246 bytes
 .../resources/access-control/localhost-ts.jks   |  Bin 0 -> 957 bytes
 .../resources/access-control/nifi.properties    |  136 +
 .../access-control/reporting-tasks.xml          |   17 +
 .../src/test/resources/logback-test.xml         |   36 +
 .../framework/web/nifi-web-docs/.gitignore      |    1 +
 .../framework/web/nifi-web-docs/pom.xml         |   57 +
 .../nifi/web/docs/DocumentationController.java  |   91 +
 .../main/webapp/WEB-INF/jsp/documentation.jsp   |  130 +
 .../WEB-INF/jsp/no-documentation-found.jsp      |   31 +
 .../src/main/webapp/WEB-INF/web.xml             |   33 +
 .../src/main/webapp/css/component-usage.css     |  127 +
 .../nifi-web-docs/src/main/webapp/css/main.css  |  214 ++
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 0 -> 189 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 0 -> 1697 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 0 -> 232 bytes
 .../src/main/webapp/js/application.js           |  282 ++
 .../framework/web/nifi-web-error/.gitignore     |    1 +
 .../framework/web/nifi-web-error/pom.xml        |   48 +
 .../apache/nifi/web/filter/CatchAllFilter.java  |   45 +
 .../src/main/webapp/WEB-INF/web.xml             |   26 +
 .../nifi-web-error/src/main/webapp/index.jsp    |   37 +
 .../framework/web/nifi-web-ui/.gitignore        |    1 +
 .../framework/web/nifi-web-ui/pom.xml           |  648 ++++
 .../apache/nifi/web/filter/IeEdgeHeader.java    |   57 +
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 +
 .../filters/bulletin-board-min.properties       |   18 +
 .../resources/filters/bulletin-board.properties |   25 +
 .../resources/filters/canvas-min.properties     |   18 +
 .../main/resources/filters/canvas.properties    |   69 +
 .../resources/filters/cluster-min.properties    |   18 +
 .../main/resources/filters/cluster.properties   |   26 +
 .../resources/filters/counters-min.properties   |   18 +
 .../main/resources/filters/counters.properties  |   26 +
 .../resources/filters/history-min.properties    |   18 +
 .../main/resources/filters/history.properties   |   27 +
 .../resources/filters/provenance-min.properties |   18 +
 .../resources/filters/provenance.properties     |   26 +
 .../resources/filters/summary-min.properties    |   18 +
 .../main/resources/filters/summary.properties   |   32 +
 .../resources/filters/templates-min.properties  |   18 +
 .../main/resources/filters/templates.properties |   26 +
 .../main/resources/filters/users-min.properties |   18 +
 .../src/main/resources/filters/users.properties |   26 +
 .../webapp/WEB-INF/pages/bulletin-board.jsp     |   50 +
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |  118 +
 .../src/main/webapp/WEB-INF/pages/cluster.jsp   |   62 +
 .../src/main/webapp/WEB-INF/pages/counters.jsp  |   61 +
 .../src/main/webapp/WEB-INF/pages/history.jsp   |   65 +
 .../main/webapp/WEB-INF/pages/message-page.jsp  |   34 +
 .../main/webapp/WEB-INF/pages/provenance.jsp    |   68 +
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   80 +
 .../src/main/webapp/WEB-INF/pages/templates.jsp |   63 +
 .../src/main/webapp/WEB-INF/pages/users.jsp     |   71 +
 .../webapp/WEB-INF/partials/banners-main.jsp    |   19 +
 .../webapp/WEB-INF/partials/banners-utility.jsp |   19 +
 .../bulletin-board/bulletin-board-content.jsp   |   40 +
 .../WEB-INF/partials/canvas/about-dialog.jsp    |   29 +
 .../WEB-INF/partials/canvas/canvas-header.jsp   |   56 +
 .../canvas/connection-configuration.jsp         |  180 +
 .../partials/canvas/connections-dialog.jsp      |   35 +
 .../partials/canvas/fill-color-dialog.jsp       |   42 +
 .../WEB-INF/partials/canvas/flow-status.jsp     |   55 +
 .../canvas/instantiate-template-dialog.jsp      |   22 +
 .../partials/canvas/label-configuration.jsp     |   38 +
 .../WEB-INF/partials/canvas/navigation.jsp      |   57 +
 .../WEB-INF/partials/canvas/new-port-dialog.jsp |   27 +
 .../canvas/new-process-group-dialog.jsp         |   27 +
 .../partials/canvas/new-processor-dialog.jsp    |   50 +
 .../canvas/new-processor-property-dialog.jsp    |   34 +
 .../canvas/new-remote-process-group-dialog.jsp  |   27 +
 .../partials/canvas/new-template-dialog.jsp     |   33 +
 .../partials/canvas/port-configuration.jsp      |   54 +
 .../WEB-INF/partials/canvas/port-details.jsp    |   39 +
 .../canvas/process-group-configuration.jsp      |   34 +
 .../partials/canvas/process-group-details.jsp   |   33 +
 .../partials/canvas/processor-configuration.jsp |  213 ++
 .../WEB-INF/partials/canvas/registration.jsp    |   44 +
 .../canvas/remote-port-configuration.jsp        |   42 +
 .../remote-process-group-configuration.jsp      |   50 +
 .../canvas/remote-process-group-details.jsp     |   48 +
 .../canvas/remote-process-group-ports.jsp       |   47 +
 .../canvas/secure-port-configuration.jsp        |   82 +
 .../partials/canvas/secure-port-details.jsp     |   67 +
 .../partials/canvas/settings-content.jsp        |   60 +
 .../webapp/WEB-INF/partials/canvas/shell.jsp    |   27 +
 .../partials/cluster/cluster-content.jsp        |   39 +
 .../partials/cluster/node-details-dialog.jsp    |   42 +
 .../WEB-INF/partials/connection-details.jsp     |  123 +
 .../partials/counters/counters-content.jsp      |   39 +
 .../partials/history/action-details-dialog.jsp  |   27 +
 .../partials/history/history-content.jsp        |   36 +
 .../partials/history/history-filter-dialog.jsp  |   73 +
 .../partials/history/history-purge-dialog.jsp   |   42 +
 .../webapp/WEB-INF/partials/message-pane.jsp    |   23 +
 .../main/webapp/WEB-INF/partials/ok-dialog.jsp  |   22 +
 .../WEB-INF/partials/processor-details.jsp      |  158 +
 .../provenance/lineage-query-dialog.jsp         |   26 +
 .../partials/provenance/provenance-content.jsp  |   73 +
 .../provenance-event-details-dialog.jsp         |  196 +
 .../provenance/provenance-query-dialog.jsp      |   26 +
 .../provenance/provenance-search-dialog.jsp     |  101 +
 .../WEB-INF/partials/status-history-dialog.jsp  |   37 +
 .../cluster-connection-summary-dialog.jsp       |   36 +
 .../cluster-input-port-summary-dialog.jsp       |   36 +
 .../cluster-output-port-summary-dialog.jsp      |   36 +
 .../cluster-processor-summary-dialog.jsp        |   36 +
 ...ster-remote-process-group-summary-dialog.jsp |   36 +
 .../partials/summary/summary-content.jsp        |   64 +
 .../summary/system-diagnostics-dialog.jsp       |  131 +
 .../summary/view-single-node-dialog.jsp         |   24 +
 .../partials/templates/templates-content.jsp    |   59 +
 .../partials/users/group-revoke-dialog.jsp      |   22 +
 .../partials/users/group-roles-dialog.jsp       |   52 +
 .../partials/users/user-delete-dialog.jsp       |   23 +
 .../partials/users/user-details-dialog.jsp      |   56 +
 .../partials/users/user-group-dialog.jsp        |   27 +
 .../partials/users/user-revoke-dialog.jsp       |   23 +
 .../partials/users/user-roles-dialog.jsp        |   60 +
 .../WEB-INF/partials/users/users-content.jsp    |   46 +
 .../webapp/WEB-INF/partials/yes-no-dialog.jsp   |   22 +
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |  120 +
 .../nifi-web-ui/src/main/webapp/css/about.css   |   54 +
 .../nifi-web-ui/src/main/webapp/css/banner.css  |   80 +
 .../src/main/webapp/css/bulletin-board.css      |  238 ++
 .../nifi-web-ui/src/main/webapp/css/canvas.css  |   41 +
 .../nifi-web-ui/src/main/webapp/css/cluster.css |  152 +
 .../webapp/css/connection-configuration.css     |  168 +
 .../src/main/webapp/css/connection-details.css  |   77 +
 .../src/main/webapp/css/counters.css            |  134 +
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |  351 ++
 .../src/main/webapp/css/flow-status.css         |  110 +
 .../nifi-web-ui/src/main/webapp/css/graph.css   |  262 ++
 .../nifi-web-ui/src/main/webapp/css/header.css  |  683 ++++
 .../nifi-web-ui/src/main/webapp/css/history.css |  216 ++
 .../src/main/webapp/css/label-configuration.css |   55 +
 .../nifi-web-ui/src/main/webapp/css/main.css    |  298 ++
 .../src/main/webapp/css/message-page.css        |   30 +
 .../src/main/webapp/css/message-pane.css        |   44 +
 .../src/main/webapp/css/navigation.css          |  292 ++
 .../main/webapp/css/new-processor-dialog.css    |  168 +
 .../src/main/webapp/css/port-configuration.css  |  192 +
 .../src/main/webapp/css/port-details.css        |   55 +
 .../webapp/css/process-group-configuration.css  |   40 +
 .../main/webapp/css/process-group-details.css   |   27 +
 .../main/webapp/css/processor-configuration.css |  388 ++
 .../src/main/webapp/css/processor-details.css   |  119 +
 .../src/main/webapp/css/provenance.css          |  654 ++++
 .../src/main/webapp/css/registration.css        |   53 +
 .../css/remote-process-group-configuration.css  |  233 ++
 .../nifi-web-ui/src/main/webapp/css/reset.css   |   43 +
 .../src/main/webapp/css/settings.css            |   66 +
 .../nifi-web-ui/src/main/webapp/css/shell.css   |   98 +
 .../src/main/webapp/css/status-history.css      |  199 +
 .../nifi-web-ui/src/main/webapp/css/summary.css |  768 ++++
 .../src/main/webapp/css/templates.css           |  229 ++
 .../nifi-web-ui/src/main/webapp/css/users.css   |  254 ++
 .../src/main/webapp/images/addConnect.png       |  Bin 0 -> 1996 bytes
 .../src/main/webapp/images/bg-error.png         |  Bin 0 -> 63766 bytes
 .../src/main/webapp/images/bgBannerFoot.png     |  Bin 0 -> 189 bytes
 .../src/main/webapp/images/bgBannerHead.png     |  Bin 0 -> 422 bytes
 .../src/main/webapp/images/bgBreadcrumbs.png    |  Bin 0 -> 2130 bytes
 .../src/main/webapp/images/bgButton.png         |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgButtonOver.png     |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgButtonSelected.png |  Bin 0 -> 166 bytes
 .../main/webapp/images/bgButtonSelectedOver.png |  Bin 0 -> 168 bytes
 .../src/main/webapp/images/bgContextMenu.png    |  Bin 0 -> 746 bytes
 .../src/main/webapp/images/bgControlsInset.png  |  Bin 0 -> 618 bytes
 .../src/main/webapp/images/bgHeader.png         |  Bin 0 -> 1697 bytes
 .../src/main/webapp/images/bgInputText.png      |  Bin 0 -> 139 bytes
 .../src/main/webapp/images/bgNifiLogo.png       |  Bin 0 -> 4232 bytes
 .../src/main/webapp/images/bgNifiToolbar.png    |  Bin 0 -> 2123 bytes
 .../main/webapp/images/bgNifiToolbarEdge.png    |  Bin 0 -> 541 bytes
 .../src/main/webapp/images/bgOutline.png        |  Bin 0 -> 585 bytes
 .../src/main/webapp/images/bgPanCenter.png      |  Bin 0 -> 362 bytes
 .../src/main/webapp/images/bgPanZoom.png        |  Bin 0 -> 832 bytes
 .../webapp/images/bgProcessGroupDetailsArea.png |  Bin 0 -> 6596 bytes
 .../main/webapp/images/bgProcessorStatArea.png  |  Bin 0 -> 4062 bytes
 .../images/bgRemoteProcessGroupDetailsArea.png  |  Bin 0 -> 4433 bytes
 .../src/main/webapp/images/bgSearchInput.png    |  Bin 0 -> 221 bytes
 .../src/main/webapp/images/bgShellClose.png     |  Bin 0 -> 169 bytes
 .../src/main/webapp/images/bgStatusPanel.png    |  Bin 0 -> 229 bytes
 .../src/main/webapp/images/bgTabContainer.png   |  Bin 0 -> 234 bytes
 .../src/main/webapp/images/bgTableHeader.png    |  Bin 0 -> 232 bytes
 .../main/webapp/images/bgToolbarBtnBorder.png   |  Bin 0 -> 227 bytes
 .../src/main/webapp/images/blueBtnBg-over.jpg   |  Bin 0 -> 356 bytes
 .../src/main/webapp/images/blueBtnBg.jpg        |  Bin 0 -> 356 bytes
 .../src/main/webapp/images/buttonColor.png      |  Bin 0 -> 1679 bytes
 .../src/main/webapp/images/buttonCopy.png       |  Bin 0 -> 1166 bytes
 .../src/main/webapp/images/buttonDelete.png     |  Bin 0 -> 1837 bytes
 .../src/main/webapp/images/buttonDisable.png    |  Bin 0 -> 1737 bytes
 .../src/main/webapp/images/buttonEnable.png     |  Bin 0 -> 1375 bytes
 .../src/main/webapp/images/buttonGroup.png      |  Bin 0 -> 880 bytes
 .../main/webapp/images/buttonNewProperty.png    |  Bin 0 -> 590 bytes
 .../src/main/webapp/images/buttonOutline.png    |  Bin 0 -> 495 bytes
 .../src/main/webapp/images/buttonPanDown.png    |  Bin 0 -> 554 bytes
 .../src/main/webapp/images/buttonPanLeft.png    |  Bin 0 -> 588 bytes
 .../src/main/webapp/images/buttonPanRight.png   |  Bin 0 -> 530 bytes
 .../src/main/webapp/images/buttonPanUp.png      |  Bin 0 -> 551 bytes
 .../src/main/webapp/images/buttonPaste.png      |  Bin 0 -> 1372 bytes
 .../src/main/webapp/images/buttonRefresh.png    |  Bin 0 -> 915 bytes
 .../src/main/webapp/images/buttonRun.png        |  Bin 0 -> 1163 bytes
 .../src/main/webapp/images/buttonStop.png       |  Bin 0 -> 869 bytes
 .../src/main/webapp/images/buttonTemplate.png   |  Bin 0 -> 1290 bytes
 .../src/main/webapp/images/buttonZoom100.png    |  Bin 0 -> 449 bytes
 .../src/main/webapp/images/buttonZoomFit.png    |  Bin 0 -> 499 bytes
 .../src/main/webapp/images/buttonZoomIn.png     |  Bin 0 -> 435 bytes
 .../src/main/webapp/images/buttonZoomOut.png    |  Bin 0 -> 339 bytes
 .../src/main/webapp/images/cluster.png          |  Bin 0 -> 326 bytes
 .../src/main/webapp/images/grayBtnBg-over.jpg   |  Bin 0 -> 319 bytes
 .../src/main/webapp/images/grayBtnBg.jpg        |  Bin 0 -> 317 bytes
 .../nifi-web-ui/src/main/webapp/images/grid.gif |  Bin 0 -> 135 bytes
 .../src/main/webapp/images/headerTabBg.gif      |  Bin 0 -> 131 bytes
 .../src/main/webapp/images/iconAdminUser.png    |  Bin 0 -> 1960 bytes
 .../src/main/webapp/images/iconAlert.png        |  Bin 0 -> 1396 bytes
 .../src/main/webapp/images/iconAlertDialog.png  |  Bin 0 -> 1241 bytes
 .../src/main/webapp/images/iconAutoRefresh.png  |  Bin 0 -> 3102 bytes
 .../src/main/webapp/images/iconBulletin.png     |  Bin 0 -> 1066 bytes
 .../main/webapp/images/iconBulletinBoard.png    |  Bin 0 -> 2523 bytes
 .../src/main/webapp/images/iconCenterView.png   |  Bin 0 -> 338 bytes
 .../src/main/webapp/images/iconChart.png        |  Bin 0 -> 510 bytes
 .../src/main/webapp/images/iconClose.png        |  Bin 0 -> 231 bytes
 .../src/main/webapp/images/iconCluster.png      |  Bin 0 -> 2112 bytes
 .../src/main/webapp/images/iconClusterSmall.png |  Bin 0 -> 757 bytes
 .../main/webapp/images/iconCollapseChildren.png |  Bin 0 -> 571 bytes
 .../main/webapp/images/iconCollapseParents.png  |  Bin 0 -> 645 bytes
 .../src/main/webapp/images/iconColor.png        |  Bin 0 -> 738 bytes
 .../src/main/webapp/images/iconCommit.png       |  Bin 0 -> 445 bytes
 .../src/main/webapp/images/iconConfigure.png    |  Bin 0 -> 696 bytes
 .../src/main/webapp/images/iconConnect.png      |  Bin 0 -> 589 bytes
 .../src/main/webapp/images/iconConnection.png   |  Bin 0 -> 1517 bytes
 .../src/main/webapp/images/iconCopy.png         |  Bin 0 -> 514 bytes
 .../src/main/webapp/images/iconCounter.png      |  Bin 0 -> 1369 bytes
 .../src/main/webapp/images/iconDelete.png       |  Bin 0 -> 670 bytes
 .../src/main/webapp/images/iconDetails.png      |  Bin 0 -> 549 bytes
 .../src/main/webapp/images/iconDisable.png      |  Bin 0 -> 764 bytes
 .../src/main/webapp/images/iconDisconnect.png   |  Bin 0 -> 569 bytes
 .../src/main/webapp/images/iconEdit.png         |  Bin 0 -> 493 bytes
 .../src/main/webapp/images/iconEditButton.png   |  Bin 0 -> 915 bytes
 .../main/webapp/images/iconExpandChildren.png   |  Bin 0 -> 521 bytes
 .../main/webapp/images/iconExpandParents.png    |  Bin 0 -> 585 bytes
 .../src/main/webapp/images/iconExport.png       |  Bin 0 -> 453 bytes
 .../main/webapp/images/iconExportLineage.png    |  Bin 0 -> 405 bytes
 .../src/main/webapp/images/iconFlowSettings.png |  Bin 0 -> 2624 bytes
 .../src/main/webapp/images/iconFunnel.png       |  Bin 0 -> 1223 bytes
 .../src/main/webapp/images/iconGoTo.png         |  Bin 0 -> 448 bytes
 .../src/main/webapp/images/iconHistory.png      |  Bin 0 -> 2347 bytes
 .../src/main/webapp/images/iconInfo.png         |  Bin 0 -> 550 bytes
 .../src/main/webapp/images/iconInputPort.png    |  Bin 0 -> 1842 bytes
 .../main/webapp/images/iconInputPortSmall.png   |  Bin 0 -> 532 bytes
 .../webapp/images/iconIsolatedProcessor.png     |  Bin 0 -> 1781 bytes
 .../src/main/webapp/images/iconLineage.png      |  Bin 0 -> 432 bytes
 .../src/main/webapp/images/iconLoading.gif      |  Bin 0 -> 2438 bytes
 .../src/main/webapp/images/iconNotSecure.png    |  Bin 0 -> 137 bytes
 .../src/main/webapp/images/iconOutputPort.png   |  Bin 0 -> 1658 bytes
 .../main/webapp/images/iconOutputPortSmall.png  |  Bin 0 -> 459 bytes
 .../src/main/webapp/images/iconPaste.png        |  Bin 0 -> 601 bytes
 .../src/main/webapp/images/iconPopOut.png       |  Bin 0 -> 217 bytes
 .../webapp/images/iconPortNotTransmitting.png   |  Bin 0 -> 231 bytes
 .../src/main/webapp/images/iconPortRemoved.png  |  Bin 0 -> 456 bytes
 .../main/webapp/images/iconPortTransmitting.png |  Bin 0 -> 235 bytes
 .../src/main/webapp/images/iconPrimary.png      |  Bin 0 -> 647 bytes
 .../src/main/webapp/images/iconProcessor.png    |  Bin 0 -> 1446 bytes
 .../src/main/webapp/images/iconProvenance.png   |  Bin 0 -> 1104 bytes
 .../src/main/webapp/images/iconRefresh.png      |  Bin 0 -> 492 bytes
 .../src/main/webapp/images/iconRemotePorts.png  |  Bin 0 -> 456 bytes
 .../webapp/images/iconRemoteProcessGroup.png    |  Bin 0 -> 674 bytes
 .../src/main/webapp/images/iconReporting.png    |  Bin 0 -> 803 bytes
 .../src/main/webapp/images/iconResetCounter.png |  Bin 0 -> 304 bytes
 .../src/main/webapp/images/iconResize.png       |  Bin 0 -> 142 bytes
 .../src/main/webapp/images/iconRevoke.png       |  Bin 0 -> 676 bytes
 .../src/main/webapp/images/iconRun.png          |  Bin 0 -> 538 bytes
 .../src/main/webapp/images/iconSecure.png       |  Bin 0 -> 133 bytes
 .../src/main/webapp/images/iconSmallFunnel.png  |  Bin 0 -> 455 bytes
 .../main/webapp/images/iconSmallInputPort.png   |  Bin 0 -> 216 bytes
 .../main/webapp/images/iconSmallOutputPort.png  |  Bin 0 -> 264 bytes
 .../webapp/images/iconSmallProcessGroup.png     |  Bin 0 -> 479 bytes
 .../main/webapp/images/iconSmallProcessor.png   |  Bin 0 -> 647 bytes
 .../webapp/images/iconSmallRelationship.png     |  Bin 0 -> 770 bytes
 .../images/iconSmallRemoteProcessGroup.png      |  Bin 0 -> 402 bytes
 .../src/main/webapp/images/iconStop.png         |  Bin 0 -> 402 bytes
 .../src/main/webapp/images/iconTemplates.png    |  Bin 0 -> 2659 bytes
 .../src/main/webapp/images/iconToFront.png      |  Bin 0 -> 475 bytes
 .../webapp/images/iconTransmissionActive.png    |  Bin 0 -> 1330 bytes
 .../webapp/images/iconTransmissionInactive.png  |  Bin 0 -> 1248 bytes
 .../src/main/webapp/images/iconTwistArrow.png   |  Bin 0 -> 256 bytes
 .../src/main/webapp/images/iconUndo.png         |  Bin 0 -> 642 bytes
 .../src/main/webapp/images/iconUsage.png        |  Bin 0 -> 470 bytes
 .../src/main/webapp/images/inputCheckbox.png    |  Bin 0 -> 389 bytes
 .../src/main/webapp/images/loadAnimation.gif    |  Bin 0 -> 10789 bytes
 .../src/main/webapp/images/nifi16.ico           |  Bin 0 -> 1150 bytes
 .../src/main/webapp/images/nifi_about.png       |  Bin 0 -> 16323 bytes
 .../src/main/webapp/images/panelBg.jpg          |  Bin 0 -> 312 bytes
 .../src/main/webapp/images/portRemoved.png      |  Bin 0 -> 1090 bytes
 .../src/main/webapp/images/portRunning.png      |  Bin 0 -> 337 bytes
 .../src/main/webapp/images/portStopped.png      |  Bin 0 -> 192 bytes
 .../src/main/webapp/images/spacer.png           |  Bin 0 -> 110 bytes
 .../src/main/webapp/images/starburst.png        |  Bin 0 -> 190 bytes
 .../src/main/webapp/images/tabBg.jpg            |  Bin 0 -> 320 bytes
 .../src/main/webapp/images/toolbarBg.gif        |  Bin 0 -> 45 bytes
 .../main/webapp/images/toolboxIconFunnel.png    |  Bin 0 -> 2409 bytes
 .../main/webapp/images/toolboxIconInputPort.png |  Bin 0 -> 2680 bytes
 .../src/main/webapp/images/toolboxIconLabel.png |  Bin 0 -> 2871 bytes
 .../webapp/images/toolboxIconOutputPort.png     |  Bin 0 -> 2209 bytes
 .../webapp/images/toolboxIconProcessGroup.png   |  Bin 0 -> 4176 bytes
 .../main/webapp/images/toolboxIconProcessor.png |  Bin 0 -> 4426 bytes
 .../images/toolboxIconRemoteProcessGroup.png    |  Bin 0 -> 5653 bytes
 .../main/webapp/images/toolboxIconTemplate.png  |  Bin 0 -> 2913 bytes
 .../images/transmissionSwitchDisabled.png       |  Bin 0 -> 1240 bytes
 .../webapp/images/transmissionSwitchEnabled.png |  Bin 0 -> 1600 bytes
 .../src/main/webapp/images/ungroup.png          |  Bin 0 -> 3409 bytes
 .../src/main/webapp/js/codemirror/LICENSE       |   19 +
 .../js/codemirror/addon/hint/show-hint.css      |   38 +
 .../js/codemirror/lib/codemirror-compressed.js  |   16 +
 .../webapp/js/codemirror/lib/codemirror.css     |  318 ++
 .../nifi-web-ui/src/main/webapp/js/d3/LICENSE   |   26 +
 .../nifi-web-ui/src/main/webapp/js/d3/d3.min.js |    5 +
 .../webapp/js/jquery/combo/jquery.combo.css     |   88 +
 .../main/webapp/js/jquery/combo/jquery.combo.js |  311 ++
 .../main/webapp/js/jquery/combo/menuArrow.png   |  Bin 0 -> 251 bytes
 .../webapp/js/jquery/combo/menuArrowSmall.png   |  Bin 0 -> 233 bytes
 .../main/webapp/js/jquery/jquery-2.1.1.min.js   |    4 +
 .../src/main/webapp/js/jquery/jquery.center.js  |   35 +
 .../src/main/webapp/js/jquery/jquery.count.js   |   99 +
 .../src/main/webapp/js/jquery/jquery.each.js    |   31 +
 .../main/webapp/js/jquery/jquery.ellipsis.js    |  164 +
 .../js/jquery/jquery.event.drag-2.2.min.js      |    6 +
 .../main/webapp/js/jquery/jquery.form.min.js    |   11 +
 .../src/main/webapp/js/jquery/jquery.tab.js     |   69 +
 .../js/jquery/minicolors/jquery.minicolors.css  |  259 ++
 .../jquery/minicolors/jquery.minicolors.min.js  |    9 +
 .../js/jquery/minicolors/jquery.minicolors.png  |  Bin 0 -> 77459 bytes
 .../webapp/js/jquery/modal/jquery.modal.css     |  110 +
 .../main/webapp/js/jquery/modal/jquery.modal.js |  266 ++
 .../js/jquery/nfeditor/jquery.nfeditor.css      |   73 +
 .../js/jquery/nfeditor/jquery.nfeditor.js       |  315 ++
 .../js/jquery/nfeditor/languages/nfel.css       |   51 +
 .../webapp/js/jquery/nfeditor/languages/nfel.js |  831 ++++
 .../webapp/js/jquery/qtip2/jquery.qtip.min.css  |    3 +
 .../webapp/js/jquery/qtip2/jquery.qtip.min.js   |    5 +
 .../webapp/js/jquery/slickgrid/MIT-LICENSE.txt  |   20 +
 .../js/jquery/slickgrid/css/images/collapse.gif |  Bin 0 -> 846 bytes
 .../js/jquery/slickgrid/css/images/expand.gif   |  Bin 0 -> 851 bytes
 .../js/jquery/slickgrid/css/images/sort-asc.gif |  Bin 0 -> 59 bytes
 .../jquery/slickgrid/css/images/sort-desc.gif   |  Bin 0 -> 59 bytes
 .../slickgrid/css/slick-default-theme.css       |  135 +
 .../js/jquery/slickgrid/css/slick.grid.css      |  158 +
 .../slickgrid/plugins/slick.autotooltips.js     |   83 +
 .../plugins/slick.cellrangedecorator.js         |   66 +
 .../plugins/slick.cellrangeselector.js          |  113 +
 .../plugins/slick.cellselectionmodel.js         |  154 +
 .../plugins/slick.rowselectionmodel.js          |  187 +
 .../webapp/js/jquery/slickgrid/slick.core.js    |  467 +++
 .../js/jquery/slickgrid/slick.dataview.js       | 1126 ++++++
 .../webapp/js/jquery/slickgrid/slick.editors.js |  512 +++
 .../js/jquery/slickgrid/slick.formatters.js     |   59 +
 .../webapp/js/jquery/slickgrid/slick.grid.js    | 3422 +++++++++++++++++
 .../webapp/js/jquery/tabbs/jquery.tabbs.css     |   48 +
 .../main/webapp/js/jquery/tabbs/jquery.tabbs.js |   96 +
 .../ui-smoothness/images/animated-overlay.gif   |  Bin 0 -> 1738 bytes
 .../images/ui-bg_flat_0_aaaaaa_40x100.png       |  Bin 0 -> 212 bytes
 .../images/ui-bg_flat_75_ffffff_40x100.png      |  Bin 0 -> 208 bytes
 .../images/ui-bg_glass_55_fbf9ee_1x400.png      |  Bin 0 -> 335 bytes
 .../images/ui-bg_glass_65_ffffff_1x400.png      |  Bin 0 -> 207 bytes
 .../images/ui-bg_glass_75_dadada_1x400.png      |  Bin 0 -> 262 bytes
 .../images/ui-bg_glass_75_e6e6e6_1x400.png      |  Bin 0 -> 262 bytes
 .../images/ui-bg_glass_95_fef1ec_1x400.png      |  Bin 0 -> 332 bytes
 .../ui-bg_highlight-soft_75_cccccc_1x100.png    |  Bin 0 -> 280 bytes
 .../images/ui-icons_222222_256x240.png          |  Bin 0 -> 6922 bytes
 .../images/ui-icons_2e83ff_256x240.png          |  Bin 0 -> 4549 bytes
 .../images/ui-icons_454545_256x240.png          |  Bin 0 -> 6992 bytes
 .../images/ui-icons_888888_256x240.png          |  Bin 0 -> 6999 bytes
 .../images/ui-icons_cd0a0a_256x240.png          |  Bin 0 -> 4549 bytes
 .../ui-smoothness/jquery-ui-1.10.4.min.css      |    7 +
 .../ui-smoothness/jquery-ui-1.10.4.min.js       |    7 +
 .../web/nifi-web-ui/src/main/webapp/js/json2.js |  475 +++
 .../js/nf/bulletin-board/nf-bulletin-board.js   |  422 ++
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 1150 ++++++
 .../src/main/webapp/js/nf/canvas/nf-birdseye.js |  340 ++
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  293 ++
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  174 +
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    | 1211 ++++++
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 +++++++
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   | 1530 ++++++++
 .../main/webapp/js/nf/canvas/nf-clipboard.js    |   96 +
 .../main/webapp/js/nf/canvas/nf-connectable.js  |  210 +
 .../js/nf/canvas/nf-connection-configuration.js | 1381 +++++++
 .../main/webapp/js/nf/canvas/nf-connection.js   | 1533 ++++++++
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  452 +++
 .../js/nf/canvas/nf-custom-processor-ui.js      |   43 +
 .../main/webapp/js/nf/canvas/nf-draggable.js    |  319 ++
 .../src/main/webapp/js/nf/canvas/nf-funnel.js   |  279 ++
 .../src/main/webapp/js/nf/canvas/nf-go-to.js    |  730 ++++
 .../webapp/js/nf/canvas/nf-graph-control.js     |  137 +
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |  194 +
 .../js/nf/canvas/nf-label-configuration.js      |  132 +
 .../src/main/webapp/js/nf/canvas/nf-label.js    |  559 +++
 .../js/nf/canvas/nf-port-configuration.js       |  170 +
 .../main/webapp/js/nf/canvas/nf-port-details.js |   63 +
 .../src/main/webapp/js/nf/canvas/nf-port.js     |  632 +++
 .../nf/canvas/nf-process-group-configuration.js |  107 +
 .../js/nf/canvas/nf-process-group-details.js    |   61 +
 .../webapp/js/nf/canvas/nf-process-group.js     | 1064 +++++
 .../js/nf/canvas/nf-processor-configuration.js  |  734 ++++
 .../nf-processor-property-combo-editor.js       |  175 +
 .../canvas/nf-processor-property-nfel-editor.js |  207 +
 .../js/nf/canvas/nf-processor-property-table.js |  567 +++
 .../canvas/nf-processor-property-text-editor.js |  212 +
 .../main/webapp/js/nf/canvas/nf-processor.js    |  831 ++++
 .../main/webapp/js/nf/canvas/nf-registration.js |   68 +
 .../nf-remote-process-group-configuration.js    |  124 +
 .../canvas/nf-remote-process-group-details.js   |   67 +
 .../nf/canvas/nf-remote-process-group-ports.js  |  525 +++
 .../js/nf/canvas/nf-remote-process-group.js     | 1060 +++++
 .../src/main/webapp/js/nf/canvas/nf-search.js   |  179 +
 .../nf/canvas/nf-secure-port-configuration.js   |  384 ++
 .../js/nf/canvas/nf-secure-port-details.js      |  118 +
 .../main/webapp/js/nf/canvas/nf-selectable.js   |   58 +
 .../src/main/webapp/js/nf/canvas/nf-settings.js |  146 +
 .../src/main/webapp/js/nf/canvas/nf-snippet.js  |  212 +
 .../src/main/webapp/js/nf/canvas/nf-storage.js  |  136 +
 .../webapp/js/nf/canvas/nf-toolbar-action.js    |   86 +
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  620 +++
 .../src/main/webapp/js/nf/cluster/nf-cluster.js |  154 +
 .../webapp/js/nf/counters/nf-counters-table.js  |  294 ++
 .../main/webapp/js/nf/counters/nf-counters.js   |  153 +
 .../webapp/js/nf/history/nf-history-model.js    |  218 ++
 .../webapp/js/nf/history/nf-history-table.js    |  429 +++
 .../src/main/webapp/js/nf/history/nf-history.js |  154 +
 .../src/main/webapp/js/nf/nf-client.js          |   56 +
 .../src/main/webapp/js/nf/nf-common.js          |  899 +++++
 .../main/webapp/js/nf/nf-connection-details.js  |  467 +++
 .../src/main/webapp/js/nf/nf-dialog.js          |  128 +
 .../src/main/webapp/js/nf/nf-namespace.js       |   20 +
 .../main/webapp/js/nf/nf-processor-details.js   |  586 +++
 .../src/main/webapp/js/nf/nf-shell.js           |  167 +
 .../src/main/webapp/js/nf/nf-status-history.js  | 1320 +++++++
 .../js/nf/provenance/nf-provenance-lineage.js   | 1413 +++++++
 .../js/nf/provenance/nf-provenance-table.js     | 1371 +++++++
 .../webapp/js/nf/provenance/nf-provenance.js    |  204 +
 .../webapp/js/nf/summary/nf-cluster-search.js   |  183 +
 .../webapp/js/nf/summary/nf-summary-table.js    | 2436 ++++++++++++
 .../src/main/webapp/js/nf/summary/nf-summary.js |  174 +
 .../js/nf/templates/nf-templates-table.js       |  332 ++
 .../main/webapp/js/nf/templates/nf-templates.js |  235 ++
 .../main/webapp/js/nf/users/nf-users-table.js   | 1090 ++++++
 .../src/main/webapp/js/nf/users/nf-users.js     |  149 +
 .../framework-bundle/framework/web/pom.xml      |   74 +
 .../web/web-optimistic-locking/.gitignore       |    1 +
 .../web/web-optimistic-locking/pom.xml          |   32 +
 .../apache/nifi/web/ConfigurationSnapshot.java  |   66 +
 .../nifi/web/OptimisticLockingManager.java      |   95 +
 .../web/StandardOptimisticLockingManager.java   |   77 +
 .../framework/web/web-security/.gitignore       |    1 +
 .../framework/web/web-security/pom.xml          |   81 +
 .../org/apache/nifi/web/security/DnUtils.java   |   87 +
 .../web/security/UntrustedProxyException.java   |   34 +
 .../anonymous/NiFiAnonymousUserFilter.java      |  101 +
 .../NiFiAuthenticationEntryPoint.java           |   68 +
 .../authorization/NiFiAuthorizationService.java |  173 +
 .../authorization/NodeAuthorizedUserFilter.java |  128 +
 .../nifi/web/security/user/NiFiUserDetails.java |  110 +
 .../nifi/web/security/user/NiFiUserUtils.java   |   82 +
 .../x509/SubjectDnX509PrincipalExtractor.java   |   38 +
 .../security/x509/X509AuthenticationFilter.java |  349 ++
 .../security/x509/X509CertificateExtractor.java |   52 +
 .../x509/ocsp/CertificateStatusException.java   |   42 +
 .../x509/ocsp/OcspCertificateValidator.java     |  444 +++
 .../web/security/x509/ocsp/OcspRequest.java     |   69 +
 .../nifi/web/security/x509/ocsp/OcspStatus.java |   82 +
 .../resources/nifi-web-security-context.xml     |   83 +
 .../NiFiAuthorizationServiceTest.java           |  250 ++
 .../nar-bundles/framework-bundle/nar/.gitignore |    1 +
 nifi/nar-bundles/framework-bundle/nar/pom.xml   |   66 +
 nifi/nar-bundles/framework-bundle/pom.xml       |  114 +
 .../hadoop-bundle/hdfs-processors/pom.xml       |   65 +
 .../hadoop/AbstractHadoopProcessor.java         |  221 ++
 .../hadoop/CreateHadoopSequenceFile.java        |  177 +
 ...lowFileStreamUnpackerSequenceFileWriter.java |  170 +
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  563 +++
 .../processors/hadoop/GetHDFSSequenceFile.java  |  146 +
 .../nifi/processors/hadoop/KeyValueReader.java  |  132 +
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  403 ++
 .../hadoop/SequenceFileWriterImpl.java          |  119 +
 .../hadoop/TarUnpackerSequenceFileWriter.java   |   54 +
 .../nifi/processors/hadoop/ValueReader.java     |  116 +
 .../hadoop/ZipUnpackerSequenceFileWriter.java   |   57 +
 .../hadoop/util/ByteFilteringOutputStream.java  |  163 +
 .../hadoop/util/InputStreamWritable.java        |   61 +
 .../hadoop/util/OutputStreamWritable.java       |   85 +
 .../hadoop/util/SequenceFileReader.java         |   28 +
 .../hadoop/util/SequenceFileWriter.java         |   40 +
 .../org.apache.nifi.processor.Processor         |   18 +
 .../index.html                                  |   88 +
 .../index.html                                  |  162 +
 .../index.html                                  |  150 +
 .../index.html                                  |  159 +
 .../processors/hadoop/AbstractHadoopTest.java   |   90 +
 .../nifi/processors/hadoop/GetHDFSTest.java     |  123 +
 .../nifi/processors/hadoop/PutHDFSTest.java     |  158 +
 .../hadoop/SimpleHadoopProcessor.java           |   30 +
 .../hadoop/TestCreateHadoopSequenceFile.java    |  176 +
 .../src/test/resources/core-site-broken.xml     |   25 +
 .../src/test/resources/core-site.xml            |   25 +
 .../resources/testdata/13545312236534130.tar    |  Bin 0 -> 1505280 bytes
 .../resources/testdata/13545423550275052.zip    |  Bin 0 -> 1500841 bytes
 .../resources/testdata/13545479542069498.pkg    |  Bin 0 -> 1500429 bytes
 .../src/test/resources/testdata/randombytes-1   |  Bin 0 -> 500000 bytes
 .../src/test/resources/testdata/randombytes-2   |  Bin 0 -> 500000 bytes
 .../src/test/resources/testdata/randombytes-3   |  Bin 0 -> 500000 bytes
 nifi/nar-bundles/hadoop-bundle/nar/pom.xml      |   41 +
 nifi/nar-bundles/hadoop-bundle/pom.xml          |   46 +
 .../hadoop-libraries-bundle/nar/pom.xml         |   45 +
 .../nar-bundles/hadoop-libraries-bundle/pom.xml |   36 +
 nifi/nar-bundles/jetty-bundle/pom.xml           |   82 +
 nifi/nar-bundles/kafka-bundle/kafka-nar/pom.xml |   37 +
 .../kafka-bundle/kafka-processors/pom.xml       |   76 +
 .../apache/nifi/processors/kafka/GetKafka.java  |  330 ++
 .../apache/nifi/processors/kafka/PutKafka.java  |  419 ++
 .../org.apache.nifi.processor.Processor         |   16 +
 .../index.html                                  |  173 +
 .../index.html                                  |  189 +
 .../nifi/processors/kafka/TestGetKafka.java     |  162 +
 .../nifi/processors/kafka/TestPutKafka.java     |  236 ++
 nifi/nar-bundles/kafka-bundle/pom.xml           |   35 +
 .../monitor-threshold-bundle/nar/pom.xml        |   39 +
 .../monitor-threshold-bundle/pom.xml            |   51 +
 .../monitor-threshold-bundle/processor/pom.xml  |   45 +
 .../processors/monitor/MonitorThreshold.java    |  904 +++++
 .../processors/monitor/ThresholdsParser.java    |  134 +
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  485 +++
 .../monitor/TestMonitorThreshold.java           | 1560 ++++++++
 .../processor/src/test/resources/testFile       |   23 +
 .../resources/threshold_settings_allow_0.xml    |   21 +
 .../resources/threshold_settings_allow_1.xml    |   21 +
 .../threshold_settings_allow_10_KB.xml          |   21 +
 .../threshold_settings_allow_10_KB_and_2_KB.xml |   26 +
 .../threshold_settings_allow_1669_bytes.xml     |   21 +
 .../resources/threshold_settings_allow_2.xml    |   21 +
 .../threshold_settings_allow_2_default.xml      |   21 +
 ...eshold_settings_allow_3336_bytes_default.xml |   21 +
 ...hold_settings_with_empty_attribute_value.xml |   21 +
 ...ettings_with_spaces_for_attribute_values.xml |   22 +
 .../ui/nb-configuration.xml                     |   18 +
 .../monitor-threshold-bundle/ui/pom.xml         |  134 +
 .../thresholds/ui/AttributeComparator.java      |   36 +
 .../thresholds/ui/AttributeResource.java        |  242 ++
 .../monitor/thresholds/ui/RuleComparator.java   |   46 +
 .../thresholds/ui/ThresholdResource.java        |  227 ++
 .../ui/ThresholdSettingsResource.java           |  191 +
 .../thresholds/ui/ThresholdsConfigFile.java     |  296 ++
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 +
 .../webapp/WEB-INF/jsp/attribute-add-dialog.jsp |   43 +
 .../WEB-INF/jsp/attribute-confirm-dialog.jsp    |   21 +
 .../WEB-INF/jsp/attribute-edit-dialog.jsp       |   27 +
 .../WEB-INF/jsp/attribute-filter-dialog.jsp     |   26 +
 .../main/webapp/WEB-INF/jsp/error-dialog.jsp    |   21 +
 .../ui/src/main/webapp/WEB-INF/jsp/index.jsp    |  101 +
 .../webapp/WEB-INF/jsp/threshold-add-dialog.jsp |   45 +
 .../WEB-INF/jsp/threshold-confirm-dialog.jsp    |   21 +
 .../webapp/WEB-INF/jsp/threshold-dialog.jsp     |   48 +
 .../WEB-INF/jsp/threshold-edit-dialog.jsp       |   45 +
 .../WEB-INF/jsp/threshold-filter-dialog.jsp     |   40 +
 .../ui/src/main/webapp/WEB-INF/web.xml          |   48 +
 .../ui/src/main/webapp/css/threshold_styles.css |  422 ++
 .../src/main/webapp/images/addWorksheetRow.png  |  Bin 0 -> 323 bytes
 .../ui/src/main/webapp/images/clear.png         |  Bin 0 -> 912 bytes
 .../ui/src/main/webapp/images/filter.gif        |  Bin 0 -> 623 bytes
 .../ui/src/main/webapp/images/firstPage.gif     |  Bin 0 -> 577 bytes
 .../main/webapp/images/firstPageDisabled.gif    |  Bin 0 -> 365 bytes
 .../ui/src/main/webapp/images/lastPage.gif      |  Bin 0 -> 579 bytes
 .../src/main/webapp/images/lastPageDisabled.gif |  Bin 0 -> 368 bytes
 .../ui/src/main/webapp/images/nextPage.gif      |  Bin 0 -> 570 bytes
 .../src/main/webapp/images/nextPageDisabled.gif |  Bin 0 -> 362 bytes
 .../ui/src/main/webapp/images/prevPage.gif      |  Bin 0 -> 567 bytes
 .../src/main/webapp/images/prevPageDisabled.gif |  Bin 0 -> 361 bytes
 .../main/webapp/images/removeWorksheetRow.png   |  Bin 0 -> 655 bytes
 .../ui/src/main/webapp/images/separator.gif     |  Bin 0 -> 63 bytes
 .../webapp/js/jquery/jqgrid/css/ui.jqgrid.css   |  171 +
 .../js/jquery/jqgrid/js/i18n/grid.locale-en.js  |  170 +
 .../js/jquery/jqgrid/js/jquery.jqGrid.min.js    |  545 +++
 .../ui/src/main/webapp/js/nf-common.js          |  888 +++++
 .../ui/src/main/xsd/threshold_settings.xsd      |   54 +
 .../nar/pom.xml                                 |   36 +
 .../persistent-provenance-repository/pom.xml    |   58 +
 .../nifi/provenance/IndexConfiguration.java     |  398 ++
 .../PersistentProvenanceRepository.java         | 1892 +++++++++
 .../provenance/RepositoryConfiguration.java     |  299 ++
 .../nifi/provenance/StandardRecordReader.java   |  305 ++
 .../nifi/provenance/StandardRecordWriter.java   |  237 ++
 .../provenance/expiration/ExpirationAction.java |   35 +
 .../expiration/FileRemovalAction.java           |   49 +
 .../provenance/lucene/DeleteIndexAction.java    |  139 +
 .../nifi/provenance/lucene/DocsReader.java      |  133 +
 .../nifi/provenance/lucene/FieldNames.java      |   23 +
 .../nifi/provenance/lucene/IndexSearch.java     |   84 +
 .../nifi/provenance/lucene/IndexingAction.java  |  219 ++
 .../nifi/provenance/lucene/LineageQuery.java    |  108 +
 .../nifi/provenance/lucene/LuceneUtil.java      |  141 +
 .../provenance/rollover/CompressionAction.java  |   59 +
 .../provenance/rollover/RolloverAction.java     |   35 +
 .../provenance/serialization/RecordReader.java  |   31 +
 .../provenance/serialization/RecordReaders.java |   70 +
 .../provenance/serialization/RecordWriter.java  |   85 +
 .../provenance/serialization/RecordWriters.java |   30 +
 ...he.nifi.provenance.ProvenanceEventRepository |   15 +
 .../TestPersistentProvenanceRepository.java     | 1118 ++++++
 .../pom.xml                                     |   42 +
 nifi/nar-bundles/pom.xml                        |  134 +
 .../standard-bundle/jms-processors/pom.xml      |   55 +
 .../apache/nifi/processors/jms/GetJMSQueue.java |   75 +
 .../apache/nifi/processors/jms/GetJMSTopic.java |  359 ++
 .../apache/nifi/processors/jms/JmsConsumer.java |  208 +
 .../org/apache/nifi/processors/jms/PutJMS.java  |  374 ++
 .../nifi/processors/jms/util/JmsFactory.java    |  435 +++
 .../nifi/processors/jms/util/JmsProperties.java |  176 +
 .../jms/util/WrappedMessageConsumer.java        |   77 +
 .../jms/util/WrappedMessageProducer.java        |   77 +
 .../org.apache.nifi.processor.Processor         |   17 +
 .../index.html                                  |  118 +
 .../index.html                                  |  122 +
 .../index.html                                  |  152 +
 .../nifi/processors/jms/GetJMSQueueTest.java    |  141 +
 nifi/nar-bundles/standard-bundle/nar/pom.xml    |   57 +
 nifi/nar-bundles/standard-bundle/pom.xml        |   71 +
 .../standard-ganglia-reporter/pom.xml           |   52 +
 .../ganglia/StandardGangliaReporter.java        |  262 ++
 .../org.apache.nifi.reporting.ReportingTask     |   15 +
 .../index.html                                  |   65 +
 .../standard-prioritizers/pom.xml               |   40 +
 .../prioritizer/FirstInFirstOutPrioritizer.java |   37 +
 .../NewestFlowFileFirstPrioritizer.java         |   37 +
 .../OldestFlowFileFirstPrioritizer.java         |   37 +
 .../PriorityAttributePrioritizer.java           |   87 +
 ...org.apache.nifi.flowfile.FlowFilePrioritizer |   18 +
 .../prioritizer/NewestFirstPrioritizerTest.java |   66 +
 .../prioritizer/OldestFirstPrioritizerTest.java |   66 +
 .../PriorityAttributePrioritizerTest.java       |  117 +
 .../standard-bundle/standard-processors/pom.xml |  154 +
 .../standard/Base64EncodeContent.java           |  142 +
 .../processors/standard/CompressContent.java    |  307 ++
 .../nifi/processors/standard/ControlRate.java   |  381 ++
 .../standard/ConvertCharacterSet.java           |  175 +
 .../processors/standard/DetectDuplicate.java    |  249 ++
 .../processors/standard/DistributeLoad.java     |  498 +++
 .../processors/standard/EncryptContent.java     |  263 ++
 .../standard/EvaluateRegularExpression.java     |  294 ++
 .../nifi/processors/standard/EvaluateXPath.java |  429 +++
 .../processors/standard/EvaluateXQuery.java     |  463 +++
 .../standard/ExecuteStreamCommand.java          |  358 ++
 .../processors/standard/GenerateFlowFile.java   |  164 +
 .../apache/nifi/processors/standard/GetFTP.java |   72 +
 .../nifi/processors/standard/GetFile.java       |  456 +++
 .../processors/standard/GetFileTransfer.java    |  301 ++
 .../nifi/processors/standard/GetHTTP.java       |  480 +++
 .../nifi/processors/standard/GetSFTP.java       |   92 +
 .../nifi/processors/standard/HashAttribute.java |  255 ++
 .../nifi/processors/standard/HashContent.java   |  151 +
 .../processors/standard/IdentifyMimeType.java   |  474 +++
 .../nifi/processors/standard/InvokeHTTP.java    |  758 ++++
 .../nifi/processors/standard/ListenHTTP.java    |  323 ++
 .../nifi/processors/standard/ListenUDP.java     |  627 +++
 .../nifi/processors/standard/LogAttribute.java  |  262 ++
 .../nifi/processors/standard/MergeContent.java  | 1015 +++++
 .../nifi/processors/standard/ModifyBytes.java   |  134 +
 .../processors/standard/MonitorActivity.java    |  206 +
 .../nifi/processors/standard/PostHTTP.java      |  904 +++++
 .../nifi/processors/standard/PutEmail.java      |  297 ++
 .../apache/nifi/processors/standard/PutFTP.java |  141 +
 .../nifi/processors/standard/PutFile.java       |  367 ++
 .../processors/standard/PutFileTransfer.java    |  293 ++
 .../nifi/processors/standard/PutSFTP.java       |   85 +
 .../nifi/processors/standard/ReplaceText.java   |  289 ++
 .../standard/ReplaceTextWithMapping.java        |  383 ++
 .../processors/standard/RouteOnAttribute.java   |  261 ++
 .../processors/standard/RouteOnContent.java     |  232 ++
 .../nifi/processors/standard/ScanAttribute.java |  229 ++
 .../nifi/processors/standard/ScanContent.java   |  292 ++
 .../processors/standard/SegmentContent.java     |  163 +
 .../nifi/processors/standard/SplitContent.java  |  260 ++
 .../nifi/processors/standard/SplitText.java     |  370 ++
 .../nifi/processors/standard/SplitXml.java      |  300 ++
 .../nifi/processors/standard/TransformXml.java  |  194 +
 .../nifi/processors/standard/UnpackContent.java |  427 +++
 .../nifi/processors/standard/ValidateXml.java   |  147 +
 .../servlets/ContentAcknowledgmentServlet.java  |  136 +
 .../standard/servlets/ListenHTTPServlet.java    |  325 ++
 .../nifi/processors/standard/util/Bin.java      |  168 +
 .../processors/standard/util/BinManager.java    |  241 ++
 .../standard/util/DocumentReaderCallback.java   |   74 +
 .../processors/standard/util/FTPTransfer.java   |  541 +++
 .../nifi/processors/standard/util/FTPUtils.java |  322 ++
 .../nifi/processors/standard/util/FileInfo.java |  167 +
 .../processors/standard/util/FileTransfer.java  |  244 ++
 .../standard/util/FlowFileSessionWrapper.java   |   44 +
 .../standard/util/NLKBufferedReader.java        |  187 +
 .../standard/util/SFTPConnection.java           |   82 +
 .../processors/standard/util/SFTPTransfer.java  |  637 +++
 .../processors/standard/util/SFTPUtils.java     |  324 ++
 .../standard/util/UDPStreamConsumer.java        |  214 ++
 .../util/ValidatingBase64InputStream.java       |   76 +
 .../standard/util/XmlElementNotifier.java       |   22 +
 .../standard/util/XmlSplitterSaxParser.java     |  112 +
 .../org.apache.nifi.processor.Processor         |   58 +
 .../index.html                                  |   63 +
 .../index.html                                  |  166 +
 .../index.html                                  |  116 +
 .../index.html                                  |   65 +
 .../index.html                                  |  147 +
 .../index.html                                  |  106 +
 .../index.html                                  |   97 +
 .../index.html                                  |  160 +
 .../index.html                                  |  135 +
 .../index.html                                  |  311 ++
 .../index.html                                  |  111 +
 .../index.html                                  |   64 +
 .../index.html                                  |  227 ++
 .../index.html                                  |  186 +
 .../index.html                                  |  143 +
 .../index.html                                  |  250 ++
 .../index.html                                  |   88 +
 .../index.html                                  |   89 +
 .../index.html                                  |  136 +
 .../index.html                                  |  181 +
 .../index.html                                  |   86 +
 .../index.html                                  |  144 +
 .../index.html                                  |   80 +
 .../index.html                                  |  347 ++
 .../index.html                                  |   64 +
 .../index.html                                  |  143 +
 .../index.html                                  |  187 +
 .../index.html                                  |  114 +
 .../index.html                                  |  283 ++
 .../index.html                                  |  109 +
 .../index.html                                  |  281 ++
 .../index.html                                  |   91 +
 .../index.html                                  |  114 +
 .../index.html                                  |  110 +
 .../index.html                                  |   82 +
 .../index.html                                  |   85 +
 .../index.html                                  |  100 +
 .../index.html                                  |  123 +
 .../index.html                                  |  107 +
 .../index.html                                  |  121 +
 .../index.html                                  |   64 +
 .../index.html                                  |   63 +
 .../index.html                                  |  163 +
 .../index.html                                  |   56 +
 .../src/test/java/TestIngestAndUpdate.java      |   34 +
 .../src/test/java/TestSuccess.java              |   24 +
 .../processors/standard/HelloWorldServlet.java  |   41 +
 .../standard/RESTServiceContentModified.java    |   78 +
 .../standard/TestBase64EncodeContent.java       |   83 +
 .../standard/TestCompressContent.java           |  111 +
 .../processors/standard/TestControlRate.java    |   68 +
 .../standard/TestConvertCharacterSet.java       |   47 +
 .../standard/TestDetectDuplicate.java           |  206 +
 .../processors/standard/TestDistributeLoad.java |  139 +
 .../processors/standard/TestEncryptContent.java |   65 +
 .../standard/TestEvaluateRegularExpression.java |  319 ++
 .../processors/standard/TestEvaluateXPath.java  |  159 +
 .../processors/standard/TestEvaluateXQuery.java |  651 ++++
 .../standard/TestExecuteStreamCommand.java      |  188 +
 .../nifi/processors/standard/TestGetFile.java   |  186 +
 .../nifi/processors/standard/TestGetHTTP.java   |  354 ++
 .../processors/standard/TestHashAttribute.java  |   99 +
 .../processors/standard/TestHashContent.java    |   67 +
 .../standard/TestIdentifyMimeType.java          |  131 +
 .../processors/standard/TestInvokeHTTP.java     |  593 +++
 .../nifi/processors/standard/TestListenUDP.java |  214 ++
 .../processors/standard/TestMergeContent.java   |  593 +++
 .../processors/standard/TestModifyBytes.java    |  190 +
 .../standard/TestMonitorActivity.java           |  193 +
 .../processors/standard/TestReplaceText.java    |  371 ++
 .../standard/TestReplaceTextLineByLine.java     |  337 ++
 .../standard/TestReplaceTextWithMapping.java    |  332 ++
 .../standard/TestRouteOnAttribute.java          |  161 +
 .../processors/standard/TestRouteOnContent.java |   73 +
 .../processors/standard/TestScanAttribute.java  |  149 +
 .../processors/standard/TestScanContent.java    |   85 +
 .../processors/standard/TestSegmentContent.java |   65 +
 .../nifi/processors/standard/TestServer.java    |  177 +
 .../processors/standard/TestSplitContent.java   |  235 ++
 .../nifi/processors/standard/TestSplitText.java |  173 +
 .../nifi/processors/standard/TestSplitXml.java  |  103 +
 .../processors/standard/TestTransformXml.java   |  120 +
 .../processors/standard/TestUnpackContent.java  |  226 ++
 .../processors/standard/TestValidateXml.java    |   42 +
 .../standard/UserAgentTestingServlet.java       |   42 +
 .../CharacterSetConversionSamples/Converted.txt |    1 +
 .../Converted2.txt                              |  Bin 0 -> 134580 bytes
 .../CharacterSetConversionSamples/Original.txt  |  418 ++
 .../resources/CompressedData/SampleFile.txt     |  418 ++
 .../resources/CompressedData/SampleFile.txt.bz2 |  Bin 0 -> 388 bytes
 .../resources/CompressedData/SampleFile.txt.gz  |  Bin 0 -> 321 bytes
 .../CompressedData/SampleFile1.txt.bz2          |  Bin 0 -> 154 bytes
 .../resources/CompressedData/SampleFile1.txt.gz |  Bin 0 -> 321 bytes
 .../CompressedData/SampleFileConcat.txt         |  835 ++++
 .../CompressedData/SampleFileConcat.txt.bz2     |  Bin 0 -> 542 bytes
 .../test/resources/ExecuteCommand/1000bytes.txt |    1 +
 .../ExecuteCommand/TestIngestAndUpdate.jar      |  Bin 0 -> 1170 bytes
 .../resources/ExecuteCommand/TestSuccess.jar    |  Bin 0 -> 827 bytes
 .../src/test/resources/ExecuteCommand/test.txt  |    2 +
 .../dictionary-with-empty-new-lines             |    5 +
 .../ScanAttribute/dictionary-with-extra-info    |    9 +
 .../test/resources/ScanAttribute/dictionary1    |    5 +
 .../test/resources/TestIdentifyMimeType/1.7z    |  Bin 0 -> 133 bytes
 .../test/resources/TestIdentifyMimeType/1.jar   |  Bin 0 -> 466 bytes
 .../test/resources/TestIdentifyMimeType/1.mdb   |  Bin 0 -> 237568 bytes
 .../test/resources/TestIdentifyMimeType/1.pdf   |  Bin 0 -> 277032 bytes
 .../test/resources/TestIdentifyMimeType/1.tar   |  Bin 0 -> 2048 bytes
 .../resources/TestIdentifyMimeType/1.txt.bz2    |  Bin 0 -> 59 bytes
 .../resources/TestIdentifyMimeType/1.txt.gz     |  Bin 0 -> 50 bytes
 .../test/resources/TestIdentifyMimeType/1.xml   |   20 +
 .../test/resources/TestIdentifyMimeType/1.zip   |  Bin 0 -> 165 bytes
 .../TestIdentifyMimeType/bgBannerFoot.png       |  Bin 0 -> 189 bytes
 .../TestIdentifyMimeType/blueBtnBg.jpg          |  Bin 0 -> 356 bytes
 .../TestIdentifyMimeType/flowfilev1.tar         |  Bin 0 -> 10240 bytes
 .../resources/TestIdentifyMimeType/flowfilev3   |  Bin 0 -> 40 bytes
 .../resources/TestIdentifyMimeType/grid.gif     |  Bin 0 -> 135 bytes
 .../test/resources/TestMergeContent/demarcate   |    1 +
 .../src/test/resources/TestMergeContent/foot    |    1 +
 .../src/test/resources/TestMergeContent/head    |    1 +
 .../test/resources/TestModifyBytes/noFooter.txt |   10 +
 .../TestModifyBytes/noFooter_noHeader.txt       |   10 +
 .../test/resources/TestModifyBytes/noHeader.txt |   11 +
 .../test/resources/TestModifyBytes/testFile.txt |   11 +
 .../TestReplaceTextLineByLine/$1$1.txt          |   11 +
 .../BRue_cRue_RiRey.txt                         |   11 +
 .../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt |   11 +
 .../TestReplaceTextLineByLine/D$d_h$d.txt       |   11 +
 .../TestReplaceTextLineByLine/Good.txt          |    1 +
 .../TestReplaceTextLineByLine/Spider.txt        |   11 +
 .../TestReplaceTextLineByLine/[DODO].txt        |   11 +
 .../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt |   11 +
 .../TestReplaceTextLineByLine/cu_Po.txt         |   11 +
 .../TestReplaceTextLineByLine/food.txt          |   11 +
 .../TestReplaceTextLineByLine/testFile.txt      |   11 +
 .../color-fruit-backreference-mapping.txt       |    7 +
 .../color-fruit-blank-mapping.txt               |    7 +
 .../color-fruit-escaped-dollar-mapping.txt      |    7 +
 ...t-excessive-backreference-mapping-simple.txt |    6 +
 ...or-fruit-excessive-backreference-mapping.txt |    6 +
 ...olor-fruit-invalid-backreference-mapping.txt |    7 +
 .../color-fruit-mapping.txt                     |    7 +
 .../color-fruit-no-match-mapping.txt            |    7 +
 .../color-fruit-space-mapping.txt               |    6 +
 .../colors-without-dashes.txt                   |    4 +
 .../TestReplaceTextWithMapping/colors.txt       |    4 +
 .../test/resources/TestScanContent/helloWorld   |    1 +
 .../resources/TestScanContent/wellthengood-bye  |    1 +
 .../src/test/resources/TestSplitText/1.txt      |    5 +
 .../src/test/resources/TestSplitText/2.txt      |    5 +
 .../src/test/resources/TestSplitText/3.txt      |    5 +
 .../src/test/resources/TestSplitText/4.txt      |    3 +
 .../src/test/resources/TestSplitText/5.txt      |    7 +
 .../src/test/resources/TestSplitText/6.txt      |    7 +
 .../test/resources/TestSplitText/original.txt   |   12 +
 .../test/resources/TestTransformXml/math.html   |    8 +
 .../test/resources/TestTransformXml/math.xml    |   21 +
 .../test/resources/TestTransformXml/math.xsl    |   36 +
 .../test/resources/TestTransformXml/tokens.csv  |    2 +
 .../test/resources/TestTransformXml/tokens.xml  |   17 +
 .../test/resources/TestTransformXml/tokens.xsl  |  103 +
 .../resources/TestUnpackContent/data.flowfilev2 |  Bin 0 -> 255 bytes
 .../resources/TestUnpackContent/data.flowfilev3 |  Bin 0 -> 357 bytes
 .../test/resources/TestUnpackContent/data.tar   |  Bin 0 -> 3584 bytes
 .../test/resources/TestUnpackContent/data.zip   |  Bin 0 -> 359 bytes
 .../resources/TestUnpackContent/folder/cal.txt  |    8 +
 .../resources/TestUnpackContent/folder/date.txt |    1 +
 .../src/test/resources/TestXml/XmlBundle.xsd    |   34 +
 .../src/test/resources/TestXml/fruit.xml        |   47 +
 .../src/test/resources/TestXml/subNode.xml      |   21 +
 .../src/test/resources/TestXml/xml-bundle-1     |   51 +
 .../src/test/resources/TestXml/xml-snippet.xml  |   25 +
 .../src/test/resources/hello.txt                |    1 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../src/test/resources/localhost.cer            |  Bin 0 -> 829 bytes
 .../src/test/resources/logback-test.xml         |   49 +
 .../standard-reporting-tasks/pom.xml            |   60 +
 .../ControllerStatusReportingTask.java          |  347 ++
 .../nifi/controller/MonitorDiskUsage.java       |  113 +
 .../apache/nifi/controller/MonitorMemory.java   |  260 ++
 .../org.apache.nifi.reporting.ReportingTask     |   17 +
 .../index.html                                  |   85 +
 .../index.html                                  |   58 +
 .../index.html                                  |   77 +
 .../nifi/controller/TestMonitorDiskUsage.java   |   61 +
 .../pom.xml                                     |   36 +
 .../distributed/cache/client/Deserializer.java  |   41 +
 .../cache/client/DistributedMapCacheClient.java |  114 +
 .../cache/client/DistributedSetCacheClient.java |   74 +
 .../distributed/cache/client/Serializer.java    |   40 +
 .../exception/DeserializationException.java     |   33 +
 .../exception/SerializationException.java       |   33 +
 .../distributed-cache-client-service/pom.xml    |   56 +
 .../distributed/cache/client/CommsSession.java  |   46 +
 .../DistributedMapCacheClientService.java       |  305 ++
 .../DistributedSetCacheClientService.java       |  215 ++
 .../cache/client/SSLCommsSession.java           |  106 +
 .../cache/client/StandardCommsSession.java      |  124 +
 ...org.apache.nifi.controller.ControllerService |   16 +
 .../index.html                                  |   78 +
 .../index.html                                  |   51 +
 .../distributed-cache-protocol/pom.xml          |   39 +
 .../cache/protocol/ProtocolHandshake.java       |  119 +
 .../protocol/exception/HandshakeException.java  |   27 +
 .../distributed-cache-server/pom.xml            |   75 +
 .../cache/server/AbstractCacheServer.java       |  199 +
 .../distributed/cache/server/CacheRecord.java   |   57 +
 .../distributed/cache/server/CacheServer.java   |   26 +
 .../cache/server/DistributedCacheServer.java    |  107 +
 .../cache/server/DistributedSetCacheServer.java |   68 +
 .../cache/server/EvictionPolicy.java            |   73 +
 .../cache/server/SetCacheServer.java            |  104 +
 .../server/map/DistributedMapCacheServer.java   |   71 +
 .../distributed/cache/server/map/MapCache.java  |   29 +
 .../cache/server/map/MapCacheRecord.java        |   58 +
 .../cache/server/map/MapCacheServer.java        |  145 +
 .../cache/server/map/MapPutResult.java          |   59 +
 .../cache/server/map/PersistentMapCache.java    |  210 +
 .../cache/server/map/SimpleMapCache.java        |  165 +
 .../cache/server/set/PersistentSetCache.java    |  194 +
 .../distributed/cache/server/set/SetCache.java  |   29 +
 .../cache/server/set/SetCacheRecord.java        |   55 +
 .../cache/server/set/SetCacheResult.java        |   43 +
 .../cache/server/set/SimpleSetCache.java        |  117 +
 ...org.apache.nifi.controller.ControllerService |   16 +
 .../index.html                                  |   82 +
 .../cache/server/TestServerAndClient.java       |  535 +++
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../distributed-cache-services-nar/pom.xml      |   47 +
 .../distributed-cache-services-bundle/pom.xml   |   35 +
 .../load-distribution-service-api/pom.xml       |   37 +
 .../nifi/loading/LoadDistributionListener.java  |   24 +
 .../nifi/loading/LoadDistributionService.java   |   33 +
 nifi/nar-bundles/standard-services/pom.xml      |   39 +
 .../ssl-context-bundle/nar/pom.xml              |   39 +
 .../ssl-context-bundle/pom.xml                  |   34 +
 .../ssl-context-service/pom.xml                 |   52 +
 .../nifi/ssl/StandardSSLContextService.java     |  354 ++
 ...org.apache.nifi.controller.ControllerService |   15 +
 .../index.html                                  |   63 +
 .../apache/nifi/ssl/SSLContextServiceTest.java  |  197 +
 .../java/org/apache/nifi/ssl/TestProcessor.java |   47 +
 .../src/test/resources/localhost-ks.jks         |  Bin 0 -> 1298 bytes
 .../src/test/resources/localhost-ts.jks         |  Bin 0 -> 891 bytes
 .../ssl-context-service-api/pom.xml             |   36 +
 .../org/apache/nifi/ssl/SSLContextService.java  |   55 +
 .../standard-services-api-nar/pom.xml           |   45 +
 .../update-attribute-bundle/model/pom.xml       |   27 +
 .../apache/nifi/update/attributes/Action.java   |   55 +
 .../nifi/update/attributes/Condition.java       |   49 +
 .../apache/nifi/update/attributes/Criteria.java |  141 +
 .../nifi/update/attributes/FlowFilePolicy.java  |   34 +
 .../org/apache/nifi/update/attributes/Rule.java |   65 +
 .../update/attributes/serde/CriteriaSerDe.java  |  128 +
 .../update-attribute-bundle/nar/pom.xml         |   48 +
 .../nar-bundles/update-attribute-bundle/pom.xml |   56 +
 .../update-attribute-bundle/processor/pom.xml   |   54 +
 .../processors/attributes/UpdateAttribute.java  |  508 +++
 .../org.apache.nifi.processor.Processor         |   15 +
 .../index.html                                  |  253 ++
 .../update/attributes/TestUpdateAttribute.java  |  425 ++
 .../ui/nb-configuration.xml                     |   18 +
 .../update-attribute-bundle/ui/pom.xml          |   91 +
 .../attributes/UpdateAttributeModelFactory.java |  127 +
 .../attributes/api/ObjectMapperResolver.java    |   51 +
 .../update/attributes/api/RuleResource.java     |  684 ++++
 .../nifi/update/attributes/dto/ActionDTO.java   |   68 +
 .../update/attributes/dto/ConditionDTO.java     |   59 +
 .../nifi/update/attributes/dto/DtoFactory.java  |   68 +
 .../nifi/update/attributes/dto/RuleDTO.java     |   65 +
 .../update/attributes/entity/ActionEntity.java  |   65 +
 .../attributes/entity/ConditionEntity.java      |   65 +
 .../entity/EvaluationContextEntity.java         |   74 +
 .../update/attributes/entity/RuleEntity.java    |   65 +
 .../update/attributes/entity/RulesEntity.java   |   66 +
 .../ui/src/main/webapp/META-INF/nifi-processor  |   15 +
 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp   |  188 +
 .../ui/src/main/webapp/WEB-INF/web.xml          |   46 +
 .../ui/src/main/webapp/css/main.css             |  426 +++
 .../ui/src/main/webapp/images/bgInputText.png   |  Bin 0 -> 139 bytes
 .../ui/src/main/webapp/images/buttonNew.png     |  Bin 0 -> 590 bytes
 .../ui/src/main/webapp/images/iconDelete.png    |  Bin 0 -> 670 bytes
 .../ui/src/main/webapp/images/iconInfo.png      |  Bin 0 -> 550 bytes
 .../ui/src/main/webapp/js/application.js        | 1915 +++++++++
 .../nar/pom.xml                                 |   36 +
 .../pom.xml                                     |   44 +
 .../volatile-provenance-repository/pom.xml      |   47 +
 .../VolatileProvenanceRepository.java           |  745 ++++
 ...he.nifi.provenance.ProvenanceEventRepository |   15 +
 .../TestVolatileProvenanceRepository.java       |  178 +
 .../src/test/resources/nifi.properties          |   18 +
 nifi/nifi-api/.gitignore                        |    2 +
 nifi/nifi-api/pom.xml                           |   31 +
 .../apache/nifi/authorization/Authority.java    |   97 +
 .../nifi/authorization/AuthorityProvider.java   |  181 +
 .../AuthorityProviderConfigurationContext.java  |   52 +
 .../AuthorityProviderInitializationContext.java |   27 +
 .../authorization/AuthorityProviderLookup.java  |   25 +
 .../authorization/DownloadAuthorization.java    |   86 +
 .../annotation/AuthorityProviderContext.java    |   36 +
 .../exception/AuthorityAccessException.java     |   33 +
 .../IdentityAlreadyExistsException.java         |   32 +
 .../exception/ProviderCreationException.java    |   40 +
 .../exception/ProviderDestructionException.java |   40 +
 .../exception/UnknownIdentityException.java     |   32 +
 .../AbstractConfigurableComponent.java          |  221 ++
 .../apache/nifi/components/AllowableValue.java  |  136 +
 .../nifi/components/ConfigurableComponent.java  |   77 +
 .../nifi/components/PropertyDescriptor.java     |  552 +++
 .../apache/nifi/components/PropertyValue.java   |  189 +
 .../nifi/components/ValidationContext.java      |   82 +
 .../nifi/components/ValidationResult.java       |  173 +
 .../org/apache/nifi/components/Validator.java   |   55 +
 .../controller/AbstractControllerService.java   |   91 +
 .../nifi/controller/ConfigurationContext.java   |   46 +
 .../nifi/controller/ControllerService.java      |  182 +
 .../ControllerServiceInitializationContext.java |   36 +
 .../controller/ControllerServiceLookup.java     |   66 +
 .../apache/nifi/controller/FlowFileQueue.java   |  200 +
 .../apache/nifi/controller/ScheduledState.java  |   37 +
 .../org/apache/nifi/controller/Snippet.java     |  113 +
 .../org/apache/nifi/controller/Triggerable.java |  116 +
 .../controller/annotation/OnConfigured.java     |   41 +
 .../repository/ContentRepository.java           |  314 ++
 .../controller/repository/FlowFileRecord.java   |   54 +
 .../repository/FlowFileRepository.java          |  128 +
 .../repository/FlowFileSwapManager.java         |   63 +
 .../controller/repository/QueueProvider.java    |   35 +
 .../controller/repository/RepositoryRecord.java |  100 +
 .../repository/RepositoryRecordType.java        |   26 +
 .../repository/claim/ContentClaim.java          |   55 +
 .../repository/claim/ContentClaimManager.java   |  143 +
 .../controller/status/ConnectionStatus.java     |  193 +
 .../nifi/controller/status/PortStatus.java      |  201 +
 .../controller/status/ProcessGroupStatus.java   |  547 +++
 .../nifi/controller/status/ProcessorStatus.java |  275 ++
 .../status/RemoteProcessGroupStatus.java        |  217 ++
 .../nifi/controller/status/RunStatus.java       |   28 +
 .../controller/status/TransmissionStatus.java   |   23 +
 .../history/ComponentStatusRepository.java      |  167 +
 .../status/history/MetricDescriptor.java        |   75 +
 .../status/history/StatusHistory.java           |   50 +
 .../status/history/StatusSnapshot.java          |   48 +
 .../controller/status/history/ValueMapper.java  |   23 +
 .../controller/status/history/ValueReducer.java |   25 +
 .../org/apache/nifi/events/EventReporter.java   |   27 +
 .../nifi/expression/AttributeExpression.java    |   79 +
 .../expression/AttributeValueDecorator.java     |   29 +
 .../expression/ExpressionLanguageCompiler.java  |   69 +
 .../java/org/apache/nifi/flowfile/FlowFile.java |  109 +
 .../nifi/flowfile/FlowFilePrioritizer.java      |   30 +
 .../java/org/apache/nifi/logging/LogLevel.java  |   27 +
 .../org/apache/nifi/logging/ProcessorLog.java   |   73 +
 .../nifi/processor/AbstractProcessor.java       |   37 +
 .../AbstractSessionFactoryProcessor.java        |  122 +
 .../org/apache/nifi/processor/DataUnit.java     |  248 ++
 .../apache/nifi/processor/FlowFileFilter.java   |   91 +
 .../apache/nifi/processor/ProcessContext.java   |  132 +
 .../apache/nifi/processor/ProcessSession.java   |  713 ++++
 .../nifi/processor/ProcessSessionFactory.java   |   30 +
 .../org/apache/nifi/processor/Processor.java    |   91 +
 .../ProcessorInitializationContext.java         |   53 +
 .../org/apache/nifi/processor/QueueSize.java    |   49 +
 .../org/apache/nifi/processor/Relationship.java |  127 +
 .../nifi/processor/SchedulingContext.java       |   70 +
 .../annotation/CapabilityDescription.java       |   39 +
 .../nifi/processor/annotation/EventDriven.java  |   49 +
 .../nifi/processor/annotation/OnAdded.java      |   41 +
 .../nifi/processor/annotation/OnRemoved.java    |   42 +
 .../nifi/processor/annotation/OnScheduled.java  |   44 +
 .../nifi/processor/annotation/OnShutdown.java   |   38 +
 .../nifi/processor/annotation/OnStopped.java    |   56 +
 .../processor/annotation/OnUnscheduled.java     |   46 +
 .../processor/annotation/SideEffectFree.java    |   45 +
 .../processor/annotation/SupportsBatching.java  |   51 +
 .../apache/nifi/processor/annotation/Tags.java  |   44 +
 .../processor/annotation/TriggerSerially.java   |   39 +
 .../TriggerWhenAnyDestinationAvailable.java     |   40 +
 .../processor/annotation/TriggerWhenEmpty.java  |   41 +
 .../exception/FlowFileAccessException.java      |   37 +
 .../exception/FlowFileHandlingException.java    |   39 +
 .../exception/MissingFlowFileException.java     |   37 +
 .../processor/exception/ProcessException.java   |   44 +
 .../nifi/processor/io/InputStreamCallback.java  |   38 +
 .../nifi/processor/io/OutputStreamCallback.java |   39 +
 .../nifi/processor/io/StreamCallback.java       |   41 +
 .../nifi/provenance/ProvenanceEventBuilder.java |  308 ++
 .../nifi/provenance/ProvenanceEventRecord.java  |  296 ++
 .../provenance/ProvenanceEventRepository.java   |  190 +
 .../nifi/provenance/ProvenanceEventType.java    |   92 +
 .../nifi/provenance/ProvenanceReporter.java     |  616 +++
 .../lineage/ComputeLineageResult.java           |   70 +
 .../lineage/ComputeLineageSubmission.java       |   82 +
 .../apache/nifi/provenance/lineage/Lineage.java |   41 +
 .../lineage/LineageComputationType.java         |   27 +
 .../nifi/provenance/lineage/LineageEdge.java    |   26 +
 .../nifi/provenance/lineage/LineageNode.java    |   61 +
 .../provenance/lineage/LineageNodeType.java     |   23 +
 .../lineage/ProvenanceEventLineageNode.java     |   32 +
 .../apache/nifi/provenance/search/Query.java    |   91 +
 .../nifi/provenance/search/QueryResult.java     |   77 +
 .../nifi/provenance/search/QuerySubmission.java |   63 +
 .../nifi/provenance/search/SearchTerm.java      |   24 +
 .../nifi/provenance/search/SearchTerms.java     |   39 +
 .../nifi/provenance/search/SearchableField.java |   62 +
 .../provenance/search/SearchableFieldType.java  |   28 +
 .../nifi/reporting/AbstractReportingTask.java   |   94 +
 .../org/apache/nifi/reporting/Bulletin.java     |  120 +
 .../apache/nifi/reporting/BulletinQuery.java    |  109 +
 .../nifi/reporting/BulletinRepository.java      |   88 +
 .../org/apache/nifi/reporting/EventAccess.java  |   53 +
 .../nifi/reporting/InitializationException.java |   32 +
 .../apache/nifi/reporting/ReportingContext.java |   99 +
 .../ReportingInitializationContext.java         |   80 +
 .../apache/nifi/reporting/ReportingTask.java    |   75 +
 .../org/apache/nifi/reporting/Severity.java     |   24 +
 .../nifi/scheduling/SchedulingStrategy.java     |   86 +
 .../org/apache/nifi/search/SearchContext.java   |   57 +
 .../org/apache/nifi/search/SearchResult.java    |   82 +
 .../java/org/apache/nifi/search/Searchable.java |   27 +
 .../nifi/web/ClusterRequestException.java       |   38 +
 .../nifi/web/InvalidRevisionException.java      |   33 +
 .../org/apache/nifi/web/NiFiWebContext.java     |  121 +
 .../apache/nifi/web/NiFiWebContextConfig.java   |   54 +
 .../nifi/web/ProcessorConfigurationAction.java  |  137 +
 .../java/org/apache/nifi/web/ProcessorInfo.java |  110 +
 .../nifi/web/ResourceNotFoundException.java     |   32 +
 .../main/java/org/apache/nifi/web/Revision.java |  110 +
 .../nifi/components/TestPropertyDescriptor.java |   59 +
 .../org/apache/nifi/processor/TestDataUnit.java |   44 +
 .../src/test/resources/logback-test.xml         |   27 +
 nifi/nifi-bootstrap/pom.xml                     |   29 +
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   92 +
 .../org/apache/nifi/bootstrap/NiFiListener.java |  128 +
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  941 +++++
 .../org/apache/nifi/bootstrap/ShutdownHook.java |   86 +
 .../exception/InvalidCommandException.java      |   37 +
 .../bootstrap/util/LimitingInputStream.java     |  107 +
 nifi/nifi-docs/pom.xml                          |  131 +
 .../src/main/asciidoc/administration-guide.adoc |  160 +
 .../src/main/asciidoc/developer-guide.adoc      |   45 +
 .../asciidoc/expression-language-guide.adoc     | 1727 +++++++++
 .../images/add-processor-with-tag-cloud.png     |  Bin 0 -> 22925 bytes
 .../src/main/asciidoc/images/add-processor.png  |  Bin 0 -> 31524 bytes
 .../src/main/asciidoc/images/addConnect.png     |  Bin 0 -> 1996 bytes
 .../src/main/asciidoc/images/comments-tab.png   |  Bin 0 -> 6431 bytes
 .../src/main/asciidoc/images/components.png     |  Bin 0 -> 8992 bytes
 .../asciidoc/images/connection-settings.png     |  Bin 0 -> 15214 bytes
 .../main/asciidoc/images/create-connection.png  |  Bin 0 -> 9430 bytes
 .../asciidoc/images/edit-property-dropdown.png  |  Bin 0 -> 14291 bytes
 .../asciidoc/images/edit-property-textarea.png  |  Bin 0 -> 17146 bytes
 .../main/asciidoc/images/event-attributes.png   |  Bin 0 -> 94668 bytes
 .../src/main/asciidoc/images/event-content.png  |  Bin 0 -> 88726 bytes
 .../src/main/asciidoc/images/event-details.png  |  Bin 0 -> 139736 bytes
 .../src/main/asciidoc/images/expand-event.png   |  Bin 0 -> 40476 bytes
 .../main/asciidoc/images/expanded-events.png    |  Bin 0 -> 76082 bytes
 .../src/main/asciidoc/images/find-parents.png   |  Bin 0 -> 35831 bytes
 .../src/main/asciidoc/images/iconAlert.png      |  Bin 0 -> 1396 bytes
 .../src/main/asciidoc/images/iconConnection.png |  Bin 0 -> 1517 bytes
 .../src/main/asciidoc/images/iconDelete.png     |  Bin 0 -> 670 bytes
 .../src/main/asciidoc/images/iconDisable.png    |  Bin 0 -> 764 bytes
 .../src/main/asciidoc/images/iconEdit.png       |  Bin 0 -> 493 bytes
 .../src/main/asciidoc/images/iconEnable.png     |  Bin 0 -> 667 bytes
 .../src/main/asciidoc/images/iconExport.png     |  Bin 0 -> 453 bytes
 .../main/asciidoc/images/iconFlowHistory.png    |  Bin 0 -> 2463 bytes
 .../src/main/asciidoc/images/iconFunnel.png     |  Bin 0 -> 1223 bytes
 .../src/main/asciidoc/images/iconInfo.png       |  Bin 0 -> 550 bytes
 .../src/main/asciidoc/images/iconInputPort.png  |  Bin 0 -> 1842 bytes
 .../main/asciidoc/images/iconInputPortSmall.png |  Bin 0 -> 532 bytes
 .../src/main/asciidoc/images/iconLabel.png      |  Bin 0 -> 838 bytes
 .../src/main/asciidoc/images/iconLineage.png    |  Bin 0 -> 2214 bytes
 .../main/asciidoc/images/iconNewTemplate.png    |  Bin 0 -> 557 bytes
 .../src/main/asciidoc/images/iconNotSecure.png  |  Bin 0 -> 221 bytes
 .../src/main/asciidoc/images/iconOutputPort.png |  Bin 0 -> 1658 bytes
 .../asciidoc/images/iconOutputPortSmall.png     |  Bin 0 -> 459 bytes
 .../main/asciidoc/images/iconProcessGroup.png   |  Bin 0 -> 1422 bytes
 .../src/main/asciidoc/images/iconProcessor.png  |  Bin 0 -> 1446 bytes
 .../src/main/asciidoc/images/iconProvenance.png |  Bin 0 -> 2268 bytes
 .../asciidoc/images/iconRemoteProcessGroup.png  |  Bin 0 -> 674 bytes
 .../src/main/asciidoc/images/iconResize.png     |  Bin 0 -> 165 bytes
 .../src/main/asciidoc/images/iconRun.png        |  Bin 0 -> 538 bytes
 .../src/main/asciidoc/images/iconSecure.png     |  Bin 0 -> 225 bytes
 .../src/main/asciidoc/images/iconSettings.png   |  Bin 0 -> 2638 bytes
 .../src/main/asciidoc/images/iconStop.png       |  Bin 0 -> 402 bytes
 .../src/main/asciidoc/images/iconSummary.png    |  Bin 0 -> 272 bytes
 .../src/main/asciidoc/images/iconTemplate.png   |  Bin 0 -> 970 bytes
 .../asciidoc/images/iconTransmissionActive.png  |  Bin 0 -> 1330 bytes
 .../images/iconTransmissionInactive.png         |  Bin 0 -> 1248 bytes
 .../src/main/asciidoc/images/iconUsers.png      |  Bin 0 -> 2272 bytes
 .../main/asciidoc/images/iconViewDetails.png    |  Bin 0 -> 1788 bytes
 .../images/instantiate-template-description.png |  Bin 0 -> 6508 bytes
 .../asciidoc/images/instantiate-template.png    |  Bin 0 -> 3216 bytes
 .../main/asciidoc/images/invalid-processor.png  |  Bin 0 -> 8187 bytes
 .../main/asciidoc/images/lineage-flowfile.png   |  Bin 0 -> 3855 bytes
 .../asciidoc/images/lineage-graph-annotated.png |  Bin 0 -> 170122 bytes
 .../src/main/asciidoc/images/new-flow.png       |  Bin 0 -> 266913 bytes
 .../main/asciidoc/images/nifi-arch-cluster.png  |  Bin 0 -> 50384 bytes
 .../src/main/asciidoc/images/nifi-arch.png      |  Bin 0 -> 43509 bytes
 .../main/asciidoc/images/nifi-navigation.png    |  Bin 0 -> 339195 bytes
 .../asciidoc/images/nifi-toolbar-components.png |  Bin 0 -> 262314 bytes
 .../src/main/asciidoc/images/parent-found.png   |  Bin 0 -> 42814 bytes
 .../asciidoc/images/process-group-anatomy.png   |  Bin 0 -> 48554 bytes
 .../main/asciidoc/images/processor-anatomy.png  |  Bin 0 -> 37035 bytes
 .../images/processor-connection-bubble.png      |  Bin 0 -> 9936 bytes
 .../src/main/asciidoc/images/properties-tab.png |  Bin 0 -> 11847 bytes
 .../asciidoc/images/provenance-annotated.png    |  Bin 0 -> 530929 bytes
 .../main/asciidoc/images/provenance-table.png   |  Bin 0 -> 466599 bytes
 .../asciidoc/images/remote-group-anatomy.png    |  Bin 0 -> 61383 bytes
 .../images/remote-group-ports-dialog.png        |  Bin 0 -> 21662 bytes
 .../images/remote-port-connection-status.png    |  Bin 0 -> 57332 bytes
 .../src/main/asciidoc/images/scheduling-tab.png |  Bin 0 -> 12288 bytes
 .../src/main/asciidoc/images/search-events.png  |  Bin 0 -> 62626 bytes
 .../images/search-receive-event-abc.png         |  Bin 0 -> 67817 bytes
 .../src/main/asciidoc/images/settings-tab.png   |  Bin 0 -> 16225 bytes
 .../src/main/asciidoc/images/simple-flow.png    |  Bin 0 -> 196808 bytes
 .../src/main/asciidoc/images/stats-history.png  |  Bin 0 -> 33205 bytes
 .../src/main/asciidoc/images/status-bar.png     |  Bin 0 -> 249077 bytes
 .../main/asciidoc/images/summary-annotated.png  |  Bin 0 -> 111956 bytes
 .../src/main/asciidoc/images/summary-table.png  |  Bin 0 -> 62114 bytes
 .../main/asciidoc/images/valid-processor.png    |  Bin 0 -> 8090 bytes
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |  296 ++
 .../nifi-docs/src/main/asciidoc/user-guide.adoc | 1286 +++++++
 .../src/main/assembly/dependencies.xml          |   28 +
 nifi/nifi-mock/pom.xml                          |   54 +
 .../MockProvenanceEventRepository.java          |  131 +
 .../apache/nifi/reporting/BulletinFactory.java  |   43 +
 .../org/apache/nifi/reporting/MockBulletin.java |   24 +
 .../util/ControllerServiceConfiguration.java    |   74 +
 .../nifi/util/MockBulletinRepository.java       |   74 +
 .../nifi/util/MockConfigurationContext.java     |   50 +
 ...kControllerServiceInitializationContext.java |   41 +
 .../nifi/util/MockControllerServiceLookup.java  |   76 +
 .../org/apache/nifi/util/MockEventAccess.java   |   70 +
 .../java/org/apache/nifi/util/MockFlowFile.java |  279 ++
 .../org/apache/nifi/util/MockFlowFileQueue.java |   85 +
 .../apache/nifi/util/MockProcessContext.java    |  284 ++
 .../apache/nifi/util/MockProcessSession.java    | 1006 +++++
 .../MockProcessorInitializationContext.java     |   74 +
 .../org/apache/nifi/util/MockProcessorLog.java  |  402 ++
 .../org/apache/nifi/util/MockPropertyValue.java |  185 +
 .../nifi/util/MockProvenanceReporter.java       |  202 +
 .../apache/nifi/util/MockReportingContext.java  |  117 +
 .../MockReportingInitializationContext.java     |   81 +
 .../apache/nifi/util/MockSessionFactory.java    |   46 +
 .../apache/nifi/util/MockValidationContext.java |   93 +
 .../org/apache/nifi/util/ReflectionUtils.java   |  149 +
 .../apache/nifi/util/SharedSessionState.java    |   72 +
 .../apache/nifi/util/SingleSessionFactory.java  |   35 +
 .../nifi/util/StandardProcessorTestRunner.java  |  492 +++
 .../java/org/apache/nifi/util/TestRunner.java   |  542 +++
 .../java/org/apache/nifi/util/TestRunners.java  |   37 +
 nifi/pom.xml                                    | 1085 ++++++
 pom.xml                                         | 1092 ------
 4901 files changed, 324985 insertions(+), 324948 deletions(-)
----------------------------------------------------------------------



[4/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
index 0000000,a755b1a..3ac55d2
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
@@@ -1,0 -1,498 +1,498 @@@
+ /*
+  * 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.processors.standard;
+ 
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.TreeSet;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.components.ValidationContext;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.components.Validator;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.loading.LoadDistributionListener;
+ import org.apache.nifi.loading.LoadDistributionService;
+ import org.apache.nifi.processor.AbstractProcessor;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessorInitializationContext;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.EventDriven;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.SideEffectFree;
+ import org.apache.nifi.processor.annotation.SupportsBatching;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable;
+ import org.apache.nifi.processor.util.StandardValidators;
+ 
+ import org.apache.commons.lang3.StringUtils;
+ 
+ @EventDriven
+ @SideEffectFree
+ @SupportsBatching
+ @TriggerWhenAnyDestinationAvailable
+ @Tags({"distribute", "load balance", "route", "round robin", "weighted"})
+ @CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution Strategy. If using the Round Robin "
+         + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties"
+         + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name "
+         + "'5' will be receive 10 FlowFiles in each iteration instead of 1.")
+ public class DistributeLoad extends AbstractProcessor {
+ 
+     public static final String STRATEGY_ROUND_ROBIN = "round robin";
+     public static final String STRATEGY_NEXT_AVAILABLE = "next available";
+     public static final String STRATEGY_LOAD_DISTRIBUTION_SERVICE = "load distribution service";
+ 
+     public static final PropertyDescriptor NUM_RELATIONSHIPS = new PropertyDescriptor.Builder()
+             .name("Number of Relationships")
+             .description("Determines the number of Relationships to which the load should be distributed")
+             .required(true)
+             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+             .defaultValue("1")
+             .build();
+     public static final PropertyDescriptor DISTRIBUTION_STRATEGY = new PropertyDescriptor.Builder()
+             .name("Distribution Strategy")
+             .description(
+                     "Determines how the load will be distributed. If using Round Robin, will not distribute any FlowFiles unless all destinations can accept FlowFiles; when using Next Available, will distribute FlowFiles as long as at least 1 destination can accept FlowFiles.")
+             .required(true)
+             .allowableValues(STRATEGY_ROUND_ROBIN, STRATEGY_NEXT_AVAILABLE, STRATEGY_LOAD_DISTRIBUTION_SERVICE)
+             .defaultValue(STRATEGY_ROUND_ROBIN)
+             .build();
+ 
+     public static final PropertyDescriptor HOSTNAMES = new PropertyDescriptor.Builder()
+             .name("Hostnames")
+             .description("List of remote servers to distribute across. Each server must be FQDN and use either ',', ';', or [space] as a delimiter")
+             .required(true)
+             .addValidator(new Validator() {
+ 
+                 @Override
+                 public ValidationResult validate(String subject, String input, ValidationContext context) {
+                     ValidationResult result = new ValidationResult.Builder()
+                     .subject(subject)
+                     .valid(true)
+                     .input(input)
+                     .explanation("Good FQDNs")
+                     .build();
+                     if (null == input) {
+                         result = new ValidationResult.Builder()
+                         .subject(subject)
+                         .input(input)
+                         .valid(false)
+                         .explanation("Need to specify delimited list of FQDNs")
+                         .build();
+                         return result;
+                     }
+                     String[] hostNames = input.split("(?:,+|;+|\\s+)");
+                     for (String hostName : hostNames) {
+                         if (StringUtils.isNotBlank(hostName) && !hostName.contains(".")) {
+                             result = new ValidationResult.Builder()
+                             .subject(subject)
+                             .input(input)
+                             .valid(false)
+                             .explanation("Need a FQDN rather than a simple host name.")
+                             .build();
+                             return result;
+                         }
+                     }
+                     return result;
+                 }
+             })
+             .build();
+     public static final PropertyDescriptor LOAD_DISTRIBUTION_SERVICE_TEMPLATE = new PropertyDescriptor.Builder()
+             .name("Load Distribution Service ID")
+             .description("The identifier of the Load Distribution Service")
+             .required(true)
+             .identifiesControllerService(LoadDistributionService.class)
+             .build();
+ 
+     private List<PropertyDescriptor> properties;
+     private final AtomicReference<Set<Relationship>> relationshipsRef = new AtomicReference<>();
+     private final AtomicReference<DistributionStrategy> strategyRef = new AtomicReference<DistributionStrategy>(new RoundRobinStrategy());
+     private final AtomicReference<List<Relationship>> weightedRelationshipListRef = new AtomicReference<>();
+     private final AtomicBoolean doCustomValidate = new AtomicBoolean(false);
+     private volatile LoadDistributionListener myListener;
+     private final AtomicBoolean doSetProps = new AtomicBoolean(true);
+ 
+     @Override
+     protected void init(final ProcessorInitializationContext context) {
+         final Set<Relationship> relationships = new HashSet<>();
+         relationships.add(createRelationship(1));
+         relationshipsRef.set(Collections.unmodifiableSet(relationships));
+ 
+         final List<PropertyDescriptor> properties = new ArrayList<>();
+         properties.add(NUM_RELATIONSHIPS);
+         properties.add(DISTRIBUTION_STRATEGY);
+         this.properties = Collections.unmodifiableList(properties);
+     }
+ 
+     private static Relationship createRelationship(final int num) {
+         return new Relationship.Builder().name(String.valueOf(num)).build();
+     }
+ 
+     @Override
+     public Set<Relationship> getRelationships() {
+         return relationshipsRef.get();
+     }
+ 
+     @Override
+     public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+         if (descriptor.equals(NUM_RELATIONSHIPS)) {
+             final Set<Relationship> relationships = new HashSet<>();
+             for (int i = 1; i <= Integer.parseInt(newValue); i++) {
+                 relationships.add(createRelationship(i));
+             }
+             this.relationshipsRef.set(Collections.unmodifiableSet(relationships));
+         } else if (descriptor.equals(DISTRIBUTION_STRATEGY)) {
+             switch (newValue.toLowerCase()) {
+                 case STRATEGY_ROUND_ROBIN:
+                     strategyRef.set(new RoundRobinStrategy());
+                     break;
+                 case STRATEGY_NEXT_AVAILABLE:
+                     strategyRef.set(new NextAvailableStrategy());
+                     break;
+                 case STRATEGY_LOAD_DISTRIBUTION_SERVICE:
+                     strategyRef.set(new LoadDistributionStrategy());
+             }
+             doSetProps.set(true);
+             doCustomValidate.set(true);
+         }
+     }
+ 
+     @Override
+     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+         if (strategyRef.get() instanceof LoadDistributionStrategy && doSetProps.getAndSet(false)) {
+             final List<PropertyDescriptor> props = new ArrayList<>(properties);
+             props.add(LOAD_DISTRIBUTION_SERVICE_TEMPLATE);
+             props.add(HOSTNAMES);
+             this.properties = Collections.unmodifiableList(props);
+         } else if (doSetProps.getAndSet(false)) {
+             final List<PropertyDescriptor> props = new ArrayList<>();
+             props.add(NUM_RELATIONSHIPS);
+             props.add(DISTRIBUTION_STRATEGY);
+             this.properties = Collections.unmodifiableList(props);
+         }
+         return properties;
+     }
+ 
+     @Override
+     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+         // validate that the property name is valid.
+         final int numRelationships = this.relationshipsRef.get().size();
+         try {
+             final int value = Integer.parseInt(propertyDescriptorName);
+             if (value <= 0 || value > numRelationships) {
+                 return new PropertyDescriptor.Builder().addValidator(new InvalidPropertyNameValidator(propertyDescriptorName))
+                         .name(propertyDescriptorName).build();
+             }
+         } catch (final NumberFormatException e) {
+             return new PropertyDescriptor.Builder().addValidator(new InvalidPropertyNameValidator(propertyDescriptorName))
+                     .name(propertyDescriptorName).build();
+         }
+ 
+         // validate that the property value is valid
+         return new PropertyDescriptor.Builder().addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+                 .name(propertyDescriptorName).dynamic(true).build();
+     }
+ 
+     @Override
+     protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+         Collection<ValidationResult> results = new ArrayList<>();
+         if (doCustomValidate.getAndSet(false)) {
+             String distStrat = validationContext.getProperty(DISTRIBUTION_STRATEGY).getValue();
+             if (distStrat.equals(STRATEGY_LOAD_DISTRIBUTION_SERVICE)) {
+                 // make sure Hostnames and Controller service are set
+                 PropertyValue propDesc = validationContext.getProperty(HOSTNAMES);
+                 if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) {
+                     results.add(new ValidationResult.Builder()
+                             .subject(HOSTNAMES.getName())
+                             .explanation("Must specify Hostnames when using 'Load Distribution Strategy'")
+                             .valid(false)
+                             .build());
+                 }
+                 propDesc = validationContext.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE);
+                 if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) {
+                     results.add(new ValidationResult.Builder()
+                             .subject(LOAD_DISTRIBUTION_SERVICE_TEMPLATE.getName())
+                             .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution Strategy'")
+                             .valid(false)
+                             .build());
+                 }
+                 if (results.isEmpty()) {
+                     int numRels = validationContext.getProperty(NUM_RELATIONSHIPS).asInteger();
+                     String hostNamesValue = validationContext.getProperty(HOSTNAMES).getValue();
+                     String[] hostNames = hostNamesValue.split("(?:,+|;+|\\s+)");
+                     int numHosts = 0;
+                     for (String hostName : hostNames) {
+                         if (StringUtils.isNotBlank(hostName)) {
+                             hostNames[numHosts++] = hostName;
+                         }
+                     }
+                     if (numHosts > numRels) {
+                         results.add(new ValidationResult.Builder()
+                                 .subject("Number of Relationships and Hostnames")
+                                 .explanation("Number of Relationships must be equal to, or greater than, the number of host names")
+                                 .valid(false)
+                                 .build());
+                     } else {
+                         // create new relationships with descriptions of hostname
+                         Set<Relationship> relsWithDesc = new TreeSet<>();
+                         for (int i = 0; i < numHosts; i++) {
+                             relsWithDesc.add(new Relationship.Builder().name(String.valueOf(i + 1)).description(hostNames[i]).build());
+                         }
+                         // add add'l rels if configuration requires it...it probably shouldn't
+                         for (int i = numHosts + 1; i <= numRels; i++) {
+                             relsWithDesc.add(createRelationship(i));
+                         }
+                         relationshipsRef.set(Collections.unmodifiableSet(relsWithDesc));
+                     }
+                 }
+             }
+         }
+         return results;
+     }
+ 
+     @OnScheduled
+     public void createWeightedList(final ProcessContext context) {
+         final Map<Integer, Integer> weightings = new LinkedHashMap<>();
+ 
+         String distStrat = context.getProperty(DISTRIBUTION_STRATEGY).getValue();
+         if (distStrat.equals(STRATEGY_LOAD_DISTRIBUTION_SERVICE)) {
+             String hostNamesValue = context.getProperty(HOSTNAMES).getValue();
+             String[] hostNames = hostNamesValue.split("(?:,+|;+|\\s+)");
+             Set<String> hostNameSet = new HashSet<>();
+             for (String hostName : hostNames) {
+                 if (StringUtils.isNotBlank(hostName)) {
+                     hostNameSet.add(hostName);
+                 }
+             }
+             LoadDistributionService svc = context.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE).asControllerService(LoadDistributionService.class);
+             myListener = new LoadDistributionListener() {
+ 
+                 @Override
+                 public void update(Map<String, Integer> loadInfo) {
+                     for (Relationship rel : relationshipsRef.get()) {
+                         String hostname = rel.getDescription();
+                         Integer weight = 1;
+                         if (loadInfo.containsKey(hostname)) {
+                             weight = loadInfo.get(hostname);
+                         }
+                         weightings.put(Integer.decode(rel.getName()), weight);
+                     }
+                     updateWeightedRelationships(weightings);
+                 }
+             };
+ 
+             Map<String, Integer> loadInfo = svc.getLoadDistribution(hostNameSet, myListener);
+             for (Relationship rel : relationshipsRef.get()) {
+                 String hostname = rel.getDescription();
+                 Integer weight = 1;
+                 if (loadInfo.containsKey(hostname)) {
+                     weight = loadInfo.get(hostname);
+                 }
+                 weightings.put(Integer.decode(rel.getName()), weight);
+             }
+ 
+         } else {
+             final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
+             for (int i = 1; i <= numRelationships; i++) {
+                 weightings.put(i, 1);
+             }
+             for (final PropertyDescriptor propDesc : context.getProperties().keySet()) {
+                 if (!this.properties.contains(propDesc)) {
+                     final int relationship = Integer.parseInt(propDesc.getName());
+                     final int weighting = context.getProperty(propDesc).asInteger();
+                     weightings.put(relationship, weighting);
+                 }
+             }
+         }
+         updateWeightedRelationships(weightings);
+     }
+ 
+     private void updateWeightedRelationships(final Map<Integer, Integer> weightings) {
+         final List<Relationship> relationshipList = new ArrayList<>();
+         for (final Map.Entry<Integer, Integer> entry : weightings.entrySet()) {
+             final String relationshipName = String.valueOf(entry.getKey());
+             final Relationship relationship = new Relationship.Builder().name(relationshipName).build();
+             for (int i = 0; i < entry.getValue(); i++) {
+                 relationshipList.add(relationship);
+             }
+         }
+ 
+         this.weightedRelationshipListRef.set(Collections.unmodifiableList(relationshipList));
+     }
+ 
+     @Override
+     public void onTrigger(final ProcessContext context, final ProcessSession session) {
+         final FlowFile flowFile = session.get();
+         if (flowFile == null) {
+             return;
+         }
+ 
+         final DistributionStrategy strategy = strategyRef.get();
 -        final Set<Relationship> available = session.getAvailableRelationships();
++        final Set<Relationship> available = context.getAvailableRelationships();
+         final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
+         final boolean allDestinationsAvailable = (available.size() == numRelationships);
+         if (!allDestinationsAvailable && strategy.requiresAllDestinationsAvailable()) {
+             return;
+         }
+ 
 -        final Relationship relationship = strategy.mapToRelationship(session, flowFile);
++        final Relationship relationship = strategy.mapToRelationship(context, flowFile);
+         if (relationship == null) {
+             // can't transfer the FlowFiles. Roll back and yield
+             session.rollback();
+             context.yield();
+             return;
+         }
+ 
+         session.transfer(flowFile, relationship);
+         session.getProvenanceReporter().route(flowFile, relationship);
+     }
+ 
+     private static class InvalidPropertyNameValidator implements Validator {
+ 
+         private final String propertyName;
+ 
+         public InvalidPropertyNameValidator(final String propertyName) {
+             this.propertyName = propertyName;
+         }
+ 
+         @Override
+         public ValidationResult validate(final String subject, final String input, final ValidationContext validationContext) {
+             return new ValidationResult.Builder().subject("Property Name").input(propertyName)
+                     .explanation("Property Name must be a positive integer between 1 and the number of relationships (inclusive)")
+                     .valid(false)
+                     .build();
+         }
+     }
+ 
+     /**
+      * Implementations must be thread-safe.
+      */
+     private static interface DistributionStrategy {
+ 
+         /**
+          * Returns a mapping of FlowFile to Relationship or <code>null</code> if
+          * the needed relationships are not available to accept files.
+          *
+          * @param session
+          * @param flowFiles
+          * @return
+          */
 -        Relationship mapToRelationship(ProcessSession session, FlowFile flowFile);
++        Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
+ 
+         boolean requiresAllDestinationsAvailable();
+     }
+ 
+     private class LoadDistributionStrategy implements DistributionStrategy {
+ 
+         private final AtomicLong counter = new AtomicLong(0L);
+ 
+         @Override
 -        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+             final int numRelationships = relationshipList.size();
+ 
+             // create a HashSet that contains all of the available relationships, as calling #contains on HashSet
+             // is much faster than calling it on a List
+             boolean foundFreeRelationship = false;
+             Relationship relationship = null;
+ 
+             int attempts = 0;
+             while (!foundFreeRelationship) {
+                 final long counterValue = counter.getAndIncrement();
+                 final int idx = (int) (counterValue % numRelationships);
+                 relationship = relationshipList.get(idx);
 -                foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
++                foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
+                 if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
+                     return null;
+                 }
+             }
+ 
+             return relationship;
+         }
+ 
+         @Override
+         public boolean requiresAllDestinationsAvailable() {
+             return false;
+         }
+ 
+     }
+ 
+     private class RoundRobinStrategy implements DistributionStrategy {
+ 
+         private final AtomicLong counter = new AtomicLong(0L);
+ 
+         @Override
 -        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+             final long counterValue = counter.getAndIncrement();
+             final int idx = (int) (counterValue % relationshipList.size());
+             final Relationship relationship = relationshipList.get(idx);
+             return relationship;
+         }
+ 
+         @Override
+         public boolean requiresAllDestinationsAvailable() {
+             return true;
+         }
+     }
+ 
+     private class NextAvailableStrategy implements DistributionStrategy {
+ 
+         private final AtomicLong counter = new AtomicLong(0L);
+ 
+         @Override
 -        public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++        public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+             final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+             final int numRelationships = relationshipList.size();
+ 
+             // create a HashSet that contains all of the available relationships, as calling #contains on HashSet
+             // is much faster than calling it on a List
+             boolean foundFreeRelationship = false;
+             Relationship relationship = null;
+ 
+             int attempts = 0;
+             while (!foundFreeRelationship) {
+                 final long counterValue = counter.getAndIncrement();
+                 final int idx = (int) (counterValue % numRelationships);
+                 relationship = relationshipList.get(idx);
 -                foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
++                foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
+                 if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
+                     return null;
+                 }
+             }
+ 
+             return relationship;
+         }
+ 
+         @Override
+         public boolean requiresAllDestinationsAvailable() {
+             return false;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 0000000,2b0b437..b7fe97a
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@@ -1,0 -1,321 +1,323 @@@
+ /*
+  * 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.processors.standard;
+ 
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.regex.Pattern;
+ 
+ import javax.servlet.Servlet;
+ import javax.ws.rs.Path;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.stream.io.LeakyBucketStreamThrottler;
+ import org.apache.nifi.stream.io.StreamThrottler;
+ import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.ProcessorInitializationContext;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.util.StandardValidators;
+ import org.apache.nifi.processors.standard.servlets.ContentAcknowledgmentServlet;
+ import org.apache.nifi.processors.standard.servlets.ListenHTTPServlet;
+ import org.apache.nifi.ssl.SSLContextService;
+ 
+ import org.eclipse.jetty.server.Connector;
+ import org.eclipse.jetty.server.HttpConfiguration;
+ import org.eclipse.jetty.server.HttpConnectionFactory;
+ import org.eclipse.jetty.server.SecureRequestCustomizer;
+ import org.eclipse.jetty.server.Server;
+ import org.eclipse.jetty.server.ServerConnector;
+ import org.eclipse.jetty.server.SslConnectionFactory;
+ import org.eclipse.jetty.servlet.ServletContextHandler;
+ import org.eclipse.jetty.util.ssl.SslContextFactory;
+ import org.eclipse.jetty.util.thread.QueuedThreadPool;
+ 
+ @Tags({"ingest", "http", "https", "rest", "listen"})
+ @CapabilityDescription("Starts an HTTP Server that is used to receive FlowFiles from remote sources. The URL of the Service will be http://{hostname}:{port}/contentListener")
+ public class ListenHTTP extends AbstractSessionFactoryProcessor {
+ 
+     private Set<Relationship> relationships;
+     private List<PropertyDescriptor> properties;
+ 
+     public static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+             .name("success")
+             .description("Relationship for successfully received FlowFiles")
+             .build();
+ 
+     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+             .name("Listening Port")
+             .description("The Port to listen on for incoming connections")
+             .required(true)
+             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+             .build();
+     public static final PropertyDescriptor AUTHORIZED_DN_PATTERN = new PropertyDescriptor.Builder()
+             .name("Authorized DN Pattern")
+             .description("A Regular Expression to apply against the Distinguished Name of incoming connections. If the Pattern does not match the DN, the connection will be refused.")
+             .required(true)
+             .defaultValue(".*")
+             .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+             .build();
+     public static final PropertyDescriptor MAX_UNCONFIRMED_TIME = new PropertyDescriptor.Builder()
+             .name("Max Unconfirmed Flowfile Time")
+             .description("The maximum amount of time to wait for a FlowFile to be confirmed before it is removed from the cache")
+             .required(true)
+             .defaultValue("60 secs")
+             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+             .build();
+     public static final PropertyDescriptor MAX_DATA_RATE = new PropertyDescriptor.Builder()
+             .name("Max Data to Receive per Second")
+             .description("The maximum amount of data to receive per second; this allows the bandwidth to be throttled to a specified data rate; if not specified, the data rate is not throttled")
+             .required(false)
+             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+             .build();
+     public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+             .name("SSL Context Service")
+             .description("The Controller Service to use in order to obtain an SSL Context")
+             .required(false)
+             .identifiesControllerService(SSLContextService.class)
+             .build();
+     public static final PropertyDescriptor HEADERS_AS_ATTRIBUTES_REGEX = new PropertyDescriptor.Builder()
+ 		    .name("HTTP Headers to receive as Attributes (Regex)")
+ 		    .description("Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes")
+ 		    .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+ 		    .required(false)
+ 		    .build();
+ 
+     public static final String URI = "/contentListener";
+     public static final String CONTEXT_ATTRIBUTE_PROCESSOR = "processor";
+     public static final String CONTEXT_ATTRIBUTE_LOGGER = "logger";
+     public static final String CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER = "sessionFactoryHolder";
++    public static final String CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER = "processContextHolder";
+     public static final String CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN = "authorityPattern";
+     public static final String CONTEXT_ATTRIBUTE_HEADER_PATTERN = "headerPattern";
+     public static final String CONTEXT_ATTRIBUTE_FLOWFILE_MAP = "flowFileMap";
+     public static final String CONTEXT_ATTRIBUTE_STREAM_THROTTLER = "streamThrottler";
+ 
+     private volatile Server server = null;
+     private final ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap = new ConcurrentHashMap<String, FlowFileEntryTimeWrapper>();
+     private final AtomicReference<ProcessSessionFactory> sessionFactoryReference = new AtomicReference<>();
+ 
+     @Override
+     protected void init(final ProcessorInitializationContext context) {
+         final Set<Relationship> relationships = new HashSet<>();
+         relationships.add(RELATIONSHIP_SUCCESS);
+         this.relationships = Collections.unmodifiableSet(relationships);
+ 
+         final List<PropertyDescriptor> descriptors = new ArrayList<>();
+         descriptors.add(PORT);
+         descriptors.add(MAX_DATA_RATE);
+         descriptors.add(SSL_CONTEXT_SERVICE);
+         descriptors.add(AUTHORIZED_DN_PATTERN);
+         descriptors.add(MAX_UNCONFIRMED_TIME);
+         descriptors.add(HEADERS_AS_ATTRIBUTES_REGEX);
+         this.properties = Collections.unmodifiableList(descriptors);
+     }
+ 
+     @Override
+     public Set<Relationship> getRelationships() {
+         return relationships;
+     }
+ 
+     @Override
+     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+         return properties;
+     }
+ 
+     @OnStopped
+     public void shutdownHttpServer() {
+         final Server toShutdown = this.server;
+         if (toShutdown == null) {
+             return;
+         }
+ 
+         try {
+             toShutdown.stop();
+             toShutdown.destroy();
+         } catch (final Exception ex) {
+             getLogger().warn("unable to cleanly shutdown embedded server due to {}", new Object[]{ex});
+             this.server = null;
+         }
+     }
+ 
+     private void createHttpServerFromService(final ProcessContext context) throws Exception {
+         final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+         final Double maxBytesPerSecond = context.getProperty(MAX_DATA_RATE).asDataSize(DataUnit.B);
+         final StreamThrottler streamThrottler = (maxBytesPerSecond == null) ? null : new LeakyBucketStreamThrottler(maxBytesPerSecond.intValue());
+ 
+         final boolean needClientAuth = sslContextService == null ? false : sslContextService.getTrustStoreFile() != null;
+ 
+         final SslContextFactory contextFactory = new SslContextFactory();
+         contextFactory.setNeedClientAuth(needClientAuth);
+ 
+         if (needClientAuth) {
+             contextFactory.setTrustStorePath(sslContextService.getTrustStoreFile());
+             contextFactory.setTrustStoreType(sslContextService.getTrustStoreType());
+             contextFactory.setTrustStorePassword(sslContextService.getTrustStorePassword());
+         }
+ 
+         final String keystorePath = sslContextService == null ? null : sslContextService.getKeyStoreFile();
+         if (keystorePath != null) {
+             final String keystorePassword = sslContextService.getKeyStorePassword();
+             final String keyStoreType = sslContextService.getKeyStoreType();
+ 
+             contextFactory.setKeyStorePath(keystorePath);
+             contextFactory.setKeyManagerPassword(keystorePassword);
+             contextFactory.setKeyStorePassword(keystorePassword);
+             contextFactory.setKeyStoreType(keyStoreType);
+         }
+ 
+         // thread pool for the jetty instance
+         final QueuedThreadPool threadPool = new QueuedThreadPool();
+         threadPool.setName(String.format("%s (%s) Web Server", getClass().getSimpleName(), getIdentifier()));
+ 
+         // create the server instance
+         final Server server = new Server(threadPool);
+ 
+         // get the configured port
+         final int port = context.getProperty(PORT).asInteger();
+ 
+         final ServerConnector connector;
+         final HttpConfiguration httpConfiguration = new HttpConfiguration();
+         if (keystorePath == null) {
+             // create the connector
+             connector = new ServerConnector(server, new HttpConnectionFactory(httpConfiguration));
+         } else {
+             // configure the ssl connector
+             httpConfiguration.setSecureScheme("https");
+             httpConfiguration.setSecurePort(port);
+             httpConfiguration.addCustomizer(new SecureRequestCustomizer());
+ 
+             // build the connector
+             connector = new ServerConnector(server,
+                     new SslConnectionFactory(contextFactory, "http/1.1"),
+                     new HttpConnectionFactory(httpConfiguration));
+         }
+ 
+         // configure the port
+         connector.setPort(port);
+ 
+         // add the connector to the server
+         server.setConnectors(new Connector[]{connector});
+ 
+         final ServletContextHandler contextHandler = new ServletContextHandler(server, "/", true, (keystorePath != null));
+         for (final Class<? extends Servlet> cls : getServerClasses()) {
+             final Path path = cls.getAnnotation(Path.class);
+             if (path == null) {
+                 contextHandler.addServlet(cls, "/*");
+             } else {
+                 contextHandler.addServlet(cls, path.value());
+             }
+         }
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESSOR, this);
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_LOGGER, getLogger());
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER, sessionFactoryReference);
++        contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER, context);
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_FLOWFILE_MAP, flowFileMap);
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN, Pattern.compile(context.getProperty(AUTHORIZED_DN_PATTERN).getValue()));
+         contextHandler.setAttribute(CONTEXT_ATTRIBUTE_STREAM_THROTTLER, streamThrottler);
+ 
+         if (context.getProperty(HEADERS_AS_ATTRIBUTES_REGEX).isSet()) {
+         	contextHandler.setAttribute(CONTEXT_ATTRIBUTE_HEADER_PATTERN, Pattern.compile(context.getProperty(HEADERS_AS_ATTRIBUTES_REGEX).getValue()));
+         }
+         server.start();
+ 
+         this.server = server;
+     }
+ 
+     @OnScheduled
+     public void createHttpServer(final ProcessContext context) throws Exception {
+         createHttpServerFromService(context);
+     }
+ 
+     protected Set<Class<? extends Servlet>> getServerClasses() {
+         final Set<Class<? extends Servlet>> s = new HashSet<>();
+         s.add(ListenHTTPServlet.class);
+         s.add(ContentAcknowledgmentServlet.class);
+         return s;
+     }
+ 
+     private Set<String> findOldFlowFileIds(final ProcessContext ctx) {
+         final Set<String> old = new HashSet<>();
+ 
+         final long expiryMillis = ctx.getProperty(MAX_UNCONFIRMED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+         final long cutoffTime = System.currentTimeMillis() - expiryMillis;
+         for (final Map.Entry<String, FlowFileEntryTimeWrapper> entry : flowFileMap.entrySet()) {
+             final FlowFileEntryTimeWrapper wrapper = entry.getValue();
+             if (wrapper != null && wrapper.getEntryTime() < cutoffTime) {
+                 old.add(entry.getKey());
+             }
+         }
+ 
+         return old;
+     }
+ 
+     @Override
+     public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+         sessionFactoryReference.compareAndSet(null, sessionFactory);
+ 
+         for (final String id : findOldFlowFileIds(context)) {
+             final FlowFileEntryTimeWrapper wrapper = flowFileMap.remove(id);
+             if (wrapper != null) {
+                 getLogger().warn("failed to received acknowledgment for HOLD with ID {}; rolling back session", new Object[]{id});
+                 wrapper.session.rollback();
+             }
+         }
+ 
+         context.yield();
+     }
+ 
+     public static class FlowFileEntryTimeWrapper {
+ 
+         private final Set<FlowFile> flowFiles;
+         private final long entryTime;
+         private final ProcessSession session;
+ 
+         public FlowFileEntryTimeWrapper(final ProcessSession session, final Set<FlowFile> flowFiles, final long entryTime) {
+             this.flowFiles = flowFiles;
+             this.entryTime = entryTime;
+             this.session = session;
+         }
+ 
+         public Set<FlowFile> getFlowFiles() {
+             return flowFiles;
+         }
+ 
+         public long getEntryTime() {
+             return entryTime;
+         }
+ 
+         public ProcessSession getSession() {
+             return session;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
index 0000000,65b3c66..43d8395
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
@@@ -1,0 -1,627 +1,627 @@@
+ /*
+  * 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.processors.standard;
+ 
+ import java.io.IOException;
+ import java.net.InetAddress;
+ import java.net.NetworkInterface;
+ import java.net.SocketException;
+ import java.net.UnknownHostException;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Enumeration;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ import java.util.concurrent.BlockingQueue;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.LinkedBlockingQueue;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReentrantLock;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.ValidationContext;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.components.Validator;
+ import org.apache.nifi.expression.AttributeExpression;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.io.nio.BufferPool;
+ import org.apache.nifi.io.nio.ChannelListener;
+ import org.apache.nifi.io.nio.consumer.StreamConsumer;
+ import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
+ import org.apache.nifi.logging.ProcessorLog;
+ import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.OnUnscheduled;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.annotation.TriggerWhenEmpty;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.util.StandardValidators;
+ import org.apache.nifi.processors.standard.util.UDPStreamConsumer;
+ import org.apache.nifi.util.Tuple;
+ 
+ import org.apache.commons.lang3.StringUtils;
+ 
+ /**
+  * <p>
+  * This processor listens for Datagram Packets on a given port and concatenates
+  * the contents of those packets together generating flow files roughly as often
+  * as the internal buffer fills up or until no more data is currently available.
+  * </p>
+  *
+  * <p>
+  * This processor has the following required properties:
+  * <ul>
+  * <li><b>Port</b> - The port to listen on for data packets. Must be known by
+  * senders of Datagrams.</li>
+  * <li><b>Receive Timeout</b> - The time out period when waiting to receive data
+  * from the socket. Specify units. Default is 5 secs.</li>
+  * <li><b>Max Buffer Size</b> - Determines the size each receive buffer may be.
+  * Specify units. Default is 1 MB.</li>
+  * <li><b>FlowFile Size Trigger</b> - Determines the (almost) upper bound size
+  * at which a flow file would be generated. A flow file will get made even if
+  * this value isn't reached if there is no more data streaming in and this value
+  * may be exceeded by the size of a single packet. Specify units. Default is 1
+  * MB.</li>
+  * <li><b>Max size of UDP Buffer</b> - The maximum UDP buffer size that should
+  * be used. This is a suggestion to the Operating System to indicate how big the
+  * udp socket buffer should be. Specify units. Default is 1 MB.")</li>
+  * <li><b>Receive Buffer Count</b> - Number of receiving buffers to be used to
+  * accept data from the socket. Higher numbers means more ram is allocated but
+  * can allow better throughput. Default is 4.</li>
+  * <li><b>Channel Reader Interval</b> - Scheduling interval for each read
+  * channel. Specify units. Default is 50 millisecs.</li>
+  * <li><b>FlowFiles Per Session</b> - The number of flow files per session.
+  * Higher number is more efficient, but will lose more data if a problem occurs
+  * that causes a rollback of a session. Default is 10</li>
+  * </ul>
+  * </p>
+  *
+  * This processor has the following optional properties:
+  * <ul>
+  * <li><b>Sending Host</b> - IP, or name, of a remote host. Only Datagrams from
+  * the specified Sending Host Port and this host will be accepted. Improves
+  * Performance. May be a system property or an environment variable.</li>
+  * <li><b>Sending Host Port</b> - Port being used by remote host to send
+  * Datagrams. Only Datagrams from the specified Sending Host and this port will
+  * be accepted. Improves Performance. May be a system property or an environment
+  * variable.</li>
+  * </ul>
+  * </p>
+  *
+  * <p>
+  * The following relationships are required:
+  * <ul>
+  * <li><b>success</b> - Where to route newly created flow files.</li>
+  * </ul>
+  * </p>
+  *
+  */
+ @TriggerWhenEmpty
+ @Tags({"ingest", "udp", "listen", "source"})
+ @CapabilityDescription("Listens for Datagram Packets on a given port and concatenates the contents of those packets "
+         + "together generating flow files")
+ public class ListenUDP extends AbstractSessionFactoryProcessor {
+ 
+     private static final Set<Relationship> relationships;
+     private static final List<PropertyDescriptor> properties;
+ 
+     // relationships.
+     public static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+             .name("success")
+             .description("Connection which contains concatenated Datagram Packets")
+             .build();
+ 
+     static {
+         Set<Relationship> rels = new HashSet<>();
+         rels.add(RELATIONSHIP_SUCCESS);
+         relationships = Collections.unmodifiableSet(rels);
+     }
+     // required properties.
+     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+             .name("Port")
+             .description("Port to listen on. Must be known by senders of Datagrams.")
+             .addValidator(StandardValidators.PORT_VALIDATOR)
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor RECV_TIMEOUT = new PropertyDescriptor.Builder()
+             .name("Receive Timeout")
+             .description("The time out period when waiting to receive data from the socket. Specify units.")
+             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+             .defaultValue("5 secs")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
+             .name("Max Buffer Size")
+             .description("Determines the size each receive buffer may be")
+             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+             .defaultValue("1 MB")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor FLOW_FILE_SIZE_TRIGGER = new PropertyDescriptor.Builder()
+             .name("FlowFile Size Trigger")
+             .description("Determines the (almost) upper bound size at which a flow file would be generated.")
+             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+             .defaultValue("1 MB")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor MAX_UDP_BUFFER = new PropertyDescriptor.Builder()
+             .name("Max size of UDP Buffer")
+             .description("The maximum UDP buffer size that should be used. This is a suggestion to the Operating System "
+                     + "to indicate how big the udp socket buffer should be.")
+             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+             .defaultValue("1 MB")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor RECV_BUFFER_COUNT = new PropertyDescriptor.Builder()
+             .name("Receive Buffer Count")
+             .description("Number of receiving buffers to be used to accept data from the socket. Higher numbers "
+                     + "means more ram is allocated but can allow better throughput.")
+             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+             .defaultValue("4")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor CHANNEL_READER_PERIOD = new PropertyDescriptor.Builder()
+             .name("Channel Reader Interval")
+             .description("Scheduling interval for each read channel.")
+             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+             .defaultValue("50 ms")
+             .required(true)
+             .build();
+ 
+     public static final PropertyDescriptor FLOW_FILES_PER_SESSION = new PropertyDescriptor.Builder()
+             .name("FlowFiles Per Session")
+             .description("The number of flow files per session.")
+             .required(true)
+             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+             .defaultValue("10")
+             .build();
+ 
+     // optional properties.
+     public static final PropertyDescriptor SENDING_HOST = new PropertyDescriptor.Builder()
+             .name("Sending Host")
+             .description("IP, or name, of a remote host. Only Datagrams from the specified Sending Host Port and this host will "
+                     + "be accepted. Improves Performance. May be a system property or an environment variable.")
+             .addValidator(new HostValidator())
+             .expressionLanguageSupported(true)
+             .build();
+ 
+     public static final PropertyDescriptor SENDING_HOST_PORT = new PropertyDescriptor.Builder()
+             .name("Sending Host Port")
+             .description("Port being used by remote host to send Datagrams. Only Datagrams from the specified Sending Host and "
+                     + "this port will be accepted. Improves Performance. May be a system property or an environment variable.")
+             .addValidator(StandardValidators.PORT_VALIDATOR)
+             .expressionLanguageSupported(true)
+             .build();
+ 
+     private static final Set<String> interfaceSet = new HashSet<>();
+ 
+     static {
+         try {
+             final Enumeration<NetworkInterface> interfaceEnum
+                     = NetworkInterface.getNetworkInterfaces();
+             while (interfaceEnum.hasMoreElements()) {
+                 final NetworkInterface ifc = interfaceEnum.nextElement();
+                 interfaceSet.add(ifc.getName());
+             }
+         } catch (SocketException e) {
+         }
+     }
+     public static final PropertyDescriptor NETWORK_INTF_NAME = new PropertyDescriptor.Builder()
+             .name("Local Network Interface")
+             .description("The name of a local network interface to be used to restrict listening for UDP Datagrams to a specific LAN."
+                     + "May be a system property or an environment variable.")
+             .addValidator(new Validator() {
+                 @Override
+                 public ValidationResult validate(String subject, String input, ValidationContext context) {
+                     ValidationResult result = new ValidationResult.Builder()
+                     .subject("Local Network Interface")
+                     .valid(true)
+                     .input(input)
+                     .build();
+                     if (interfaceSet.contains(input.toLowerCase())) {
+                         return result;
+                     }
+ 
+                     String message;
+                     try {
+                         AttributeExpression ae = context.newExpressionLanguageCompiler().compile(input);
+                         String realValue = ae.evaluate();
+                         if (interfaceSet.contains(realValue.toLowerCase())) {
+                             return result;
+                         }
+ 
+                         message = realValue + " is not a valid network name. Valid names are " + interfaceSet.toString();
+ 
+                     } catch (IllegalArgumentException e) {
+                         message = "Not a valid AttributeExpression: " + e.getMessage();
+                     }
+                     result = new ValidationResult.Builder()
+                     .subject("Local Network Interface")
+                     .valid(false)
+                     .input(input)
+                     .explanation(message)
+                     .build();
+ 
+                     return result;
+                 }
+             })
+             .expressionLanguageSupported(true)
+             .build();
+ 
+     static {
+         List<PropertyDescriptor> props = new ArrayList<>();
+         props.add(SENDING_HOST);
+         props.add(SENDING_HOST_PORT);
+         props.add(NETWORK_INTF_NAME);
+         props.add(CHANNEL_READER_PERIOD);
+         props.add(FLOW_FILE_SIZE_TRIGGER);
+         props.add(MAX_BUFFER_SIZE);
+         props.add(MAX_UDP_BUFFER);
+         props.add(PORT);
+         props.add(RECV_BUFFER_COUNT);
+         props.add(FLOW_FILES_PER_SESSION);
+         props.add(RECV_TIMEOUT);
+         properties = Collections.unmodifiableList(props);
+     }
+     // defaults
+     public static final int DEFAULT_LISTENING_THREADS = 2;
+     // lock used to protect channelListener
+     private final Lock lock = new ReentrantLock();
+     private volatile ChannelListener channelListener = null;
+     private final BlockingQueue<Tuple<ProcessSession, List<FlowFile>>> flowFilesPerSessionQueue = new LinkedBlockingQueue<>();
+     private final List<FlowFile> newFlowFiles = new ArrayList<>();
+     private final AtomicReference<UDPStreamConsumer> consumerRef = new AtomicReference<>();
+     private final AtomicBoolean stopping = new AtomicBoolean(false);
+     private final AtomicReference<ProcessSessionFactory> sessionFactoryRef = new AtomicReference<>();
+     private final ExecutorService consumerExecutorService = Executors.newSingleThreadExecutor();
+     private final AtomicReference<Future<Tuple<ProcessSession, List<FlowFile>>>> consumerFutureRef = new AtomicReference<>();
+     private final AtomicBoolean resetChannelListener = new AtomicBoolean(false);
+     // instance attribute for provenance receive event generation
+     private volatile String sendingHost;
+ 
+     @Override
+     public Set<Relationship> getRelationships() {
+         return relationships;
+     }
+ 
+     @Override
+     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+         return properties;
+     }
+ 
+     /**
+      * Create the ChannelListener and a thread that causes the Consumer to
+      * create flow files.
+      *
+      * @param context
+      * @throws IOException
+      */
+     @OnScheduled
+     public void initializeChannelListenerAndConsumerProcessing(final ProcessContext context) throws IOException {
+         getChannelListener(context);
+         stopping.set(false);
+         Future<Tuple<ProcessSession, List<FlowFile>>> consumerFuture = consumerExecutorService
+                 .submit(new Callable<Tuple<ProcessSession, List<FlowFile>>>() {
+ 
+                     @Override
+                     public Tuple<ProcessSession, List<FlowFile>> call() {
+                         final int maxFlowFilesPerSession = context.getProperty(FLOW_FILES_PER_SESSION).asInteger();
+                         final long channelReaderIntervalMSecs = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS);
+                         // number of waits in 5 secs, or 1
+                         final int maxWaits = (int) (channelReaderIntervalMSecs <= 1000 ? 5000 / channelReaderIntervalMSecs : 1);
+                         final ProcessorLog logger = getLogger();
+                         int flowFileCount = maxFlowFilesPerSession;
+                         ProcessSession session = null;
+                         int numWaits = 0;
+                         while (!stopping.get()) {
+                             UDPStreamConsumer consumer = consumerRef.get();
+                             if (consumer == null || sessionFactoryRef.get() == null) {
+                                 try {
+                                     Thread.sleep(100L);
+                                 } catch (InterruptedException swallow) {
+                                 }
+                             } else {
+                                 try {
+                                     // first time through, flowFileCount is maxFlowFilesPerSession so that a session
+                                     // is created and the consumer is updated with it.
+                                     if (flowFileCount == maxFlowFilesPerSession || numWaits == maxWaits) {
+                                         logger.debug("Have waited {} times", new Object[]{numWaits});
+                                         numWaits = 0;
+                                         if (session != null) {
+                                             Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(
+                                                     session,
+                                                     new ArrayList<>(newFlowFiles));
+                                             newFlowFiles.clear();
+                                             flowFilesPerSessionQueue.add(flowFilesPerSession);
+                                         }
+                                         session = sessionFactoryRef.get().createSession();
+                                         consumer.setSession(session);
+                                         flowFileCount = 0;
+                                     }
+                                     // this will throttle the processing of the received datagrams. If there are no more
+                                     // buffers to read into because none have been returned to the pool via consumer.process(),
+                                     // then the desired back pressure on the channel is created.
 -                                    if (session.getAvailableRelationships().size() > 0) {
++                                    if (context.getAvailableRelationships().size() > 0) {
+                                         consumer.process();
+                                         if (flowFileCount == newFlowFiles.size()) {
+                                             // no new datagrams received, need to throttle this thread back so it does
+                                             // not consume all cpu...but don't want to cause back pressure on the channel
+                                             // so the sleep time is same as the reader interval
+                                             // If have done this for approx. 5 secs, assume datagram sender is down. So, push
+                                             // out the remaining flow files (see numWaits == maxWaits above)
+                                             Thread.sleep(channelReaderIntervalMSecs);
+                                             if (flowFileCount > 0) {
+                                                 numWaits++;
+                                             }
+                                         } else {
+                                             flowFileCount = newFlowFiles.size();
+                                         }
+                                     } else {
+                                         logger.debug("Creating back pressure...no available destinations");
+                                         Thread.sleep(1000L);
+                                     }
+                                 } catch (final IOException ioe) {
+                                     logger.error("Unable to fully process consumer {}", new Object[]{consumer}, ioe);
+                                 } catch (InterruptedException e) {
+                                     // don't care
+                                 } finally {
+                                     if (consumer.isConsumerFinished()) {
+                                         logger.info("Consumer {} was closed and is finished", new Object[]{consumer});
+                                         consumerRef.set(null);
+                                         disconnect();
+                                         if (!stopping.get()) {
+                                             resetChannelListener.set(true);
+                                         }
+                                     }
+                                 }
+                             }
+                         }
+                         // when shutting down, need consumer to drain rest of cached buffers and clean up.
+                         // prior to getting here, the channelListener was shutdown
+                         UDPStreamConsumer consumer;
+                         while ((consumer = consumerRef.get()) != null && !consumer.isConsumerFinished()) {
+                             try {
+                                 consumer.process();
+                             } catch (IOException swallow) {
+                                 // if this is blown...consumer.isConsumerFinished will be true
+                             }
+                         }
+                         Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session,
+                                 new ArrayList<>(newFlowFiles));
+                         return flowFilesPerSession;
+                     }
+                 });
+         consumerFutureRef.set(consumerFuture);
+     }
+ 
+     private void disconnect() {
+         if (lock.tryLock()) {
+             try {
+                 if (channelListener != null) {
+                     getLogger().debug("Shutting down channel listener {}", new Object[]{channelListener});
+                     channelListener.shutdown(500L, TimeUnit.MILLISECONDS);
+                     channelListener = null;
+                 }
+             } finally {
+                 lock.unlock();
+             }
+         }
+     }
+ 
+     private void getChannelListener(final ProcessContext context) throws IOException {
+         if (lock.tryLock()) {
+             try {
+                 ProcessorLog logger = getLogger();
+                 logger.debug("Instantiating a new channel listener");
+                 final int port = context.getProperty(PORT).asInteger();
+                 final int bufferCount = context.getProperty(RECV_BUFFER_COUNT).asInteger();
+                 final Double bufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B);
+                 final Double rcvBufferSize = context.getProperty(MAX_UDP_BUFFER).asDataSize(DataUnit.B);
+                 sendingHost = context.getProperty(SENDING_HOST).evaluateAttributeExpressions().getValue();
+                 final Integer sendingHostPort = context.getProperty(SENDING_HOST_PORT).evaluateAttributeExpressions().asInteger();
+                 final String nicIPAddressStr = context.getProperty(NETWORK_INTF_NAME).evaluateAttributeExpressions().getValue();
+                 final Double flowFileSizeTrigger = context.getProperty(FLOW_FILE_SIZE_TRIGGER).asDataSize(DataUnit.B);
+                 final int recvTimeoutMS = context.getProperty(RECV_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
+                 final StreamConsumerFactory consumerFactory = new StreamConsumerFactory() {
+ 
+                     @Override
+                     public StreamConsumer newInstance(final String streamId) {
+                         final UDPStreamConsumer consumer = new UDPStreamConsumer(streamId, newFlowFiles, flowFileSizeTrigger.intValue(), getLogger());
+                         consumerRef.set(consumer);
+                         return consumer;
+                     }
+                 };
+                 final int readerMilliseconds = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
+                 final BufferPool bufferPool = new BufferPool(bufferCount, bufferSize.intValue(), false, Integer.MAX_VALUE);
+                 channelListener = new ChannelListener(DEFAULT_LISTENING_THREADS, consumerFactory, bufferPool, recvTimeoutMS, TimeUnit.MILLISECONDS);
+                 // specifying a sufficiently low number for each stream to be fast enough though very efficient
+                 channelListener.setChannelReaderSchedulingPeriod(readerMilliseconds, TimeUnit.MILLISECONDS);
+                 InetAddress nicIPAddress = null;
+                 if (null != nicIPAddressStr) {
+                     NetworkInterface netIF = NetworkInterface.getByName(nicIPAddressStr);
+                     nicIPAddress = netIF.getInetAddresses().nextElement();
+                 }
+                 channelListener.addDatagramChannel(nicIPAddress, port, rcvBufferSize.intValue(), sendingHost, sendingHostPort);
+                 logger.info("Registered service and initialized UDP socket listener. Now listening on port " + port + "...");
+             } finally {
+                 lock.unlock();
+             }
+         }
+     }
+ 
+     @Override
+     protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+         Collection<ValidationResult> result = new ArrayList<>();
+         String sendingHost = validationContext.getProperty(SENDING_HOST).getValue();
+         String sendingPort = validationContext.getProperty(SENDING_HOST_PORT).getValue();
+         if (StringUtils.isBlank(sendingHost) && StringUtils.isNotBlank(sendingPort)) {
+             result.add(new ValidationResult.Builder()
+                     .subject(SENDING_HOST.getName())
+                     .valid(false)
+                     .explanation("Must specify Sending Host when specifying Sending Host Port")
+                     .build());
+         } else if (StringUtils.isBlank(sendingPort) && StringUtils.isNotBlank(sendingHost)) {
+             result.add(new ValidationResult.Builder()
+                     .subject(SENDING_HOST_PORT.getName())
+                     .valid(false)
+                     .explanation("Must specify Sending Host Port when specifying Sending Host")
+                     .build());
+         }
+         return result;
+     }
+ 
+     @Override
+     public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+         final ProcessorLog logger = getLogger();
+         sessionFactoryRef.compareAndSet(null, sessionFactory);
+         if (resetChannelListener.getAndSet(false) && !stopping.get()) {
+             try {
+                 getChannelListener(context);
+             } catch (IOException e) {
+                 logger.error("Tried to reset Channel Listener and failed due to:", e);
+                 resetChannelListener.set(true);
+             }
+         }
+ 
+         transferFlowFiles();
+     }
+ 
+     private boolean transferFlowFiles() {
+         final ProcessorLog logger = getLogger();
+         ProcessSession session;
+         Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = null;
+         boolean transferred = false;
+         try {
+             flowFilesPerSession = flowFilesPerSessionQueue.poll(100L, TimeUnit.MILLISECONDS);
+         } catch (InterruptedException e) {
+         }
+         if (flowFilesPerSession != null) {
+             session = flowFilesPerSession.getKey();
+             List<FlowFile> flowFiles = flowFilesPerSession.getValue();
+             String sourceSystem = sendingHost == null ? "Unknown" : sendingHost;
+             try {
+                 for (FlowFile flowFile : flowFiles) {
+                     session.getProvenanceReporter().receive(flowFile, sourceSystem);
+                     session.transfer(flowFile, RELATIONSHIP_SUCCESS);
+                 }
+                 logger.info("Transferred flow files {} to success", new Object[]{flowFiles});
+                 transferred = true;
+ 
+                 // need to check for erroneous flow files in input queue
+                 List<FlowFile> existingFlowFiles = session.get(10);
+                 for (FlowFile existingFlowFile : existingFlowFiles) {
+                     if (existingFlowFile != null && existingFlowFile.getSize() > 0) {
+                         session.transfer(existingFlowFile, RELATIONSHIP_SUCCESS);
+                         logger.warn("Found flow file in input queue (shouldn't have). Transferred flow file {} to success",
+                                 new Object[]{existingFlowFile});
+                     } else if (existingFlowFile != null) {
+                         session.remove(existingFlowFile);
+                         logger.warn("Found empty flow file in input queue (shouldn't have). Removed flow file {}", new Object[]{existingFlowFile});
+                     }
+                 }
+                 session.commit();
+             } catch (Throwable t) {
+                 session.rollback();
+                 logger.error("Failed to transfer flow files or commit session...rolled back", t);
+                 throw t;
+             }
+         }
+         return transferred;
+     }
+ 
+     @OnUnscheduled
+     public void stopping() {
+         getLogger().debug("Stopping Processor");
+         disconnect();
+         stopping.set(true);
+         Future<Tuple<ProcessSession, List<FlowFile>>> future;
+         Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession;
+         if ((future = consumerFutureRef.getAndSet(null)) != null) {
+             try {
+                 flowFilesPerSession = future.get();
+                 if (flowFilesPerSession.getValue().size() > 0) {
+                     getLogger().debug("Draining remaining flow Files when stopping");
+                     flowFilesPerSessionQueue.add(flowFilesPerSession);
+                 } else {
+                     // need to close out the session that has no flow files
+                     flowFilesPerSession.getKey().commit();
+                 }
+             } catch (InterruptedException | ExecutionException e) {
+                 getLogger().error("Failure in cleaning up!", e);
+             }
+             boolean moreFiles = true;
+             while (moreFiles) {
+                 try {
+                     moreFiles = transferFlowFiles();
+                 } catch (Throwable t) {
+                     getLogger().error("Problem transferring cached flowfiles", t);
+                 }
+             }
+         }
+     }
+ 
+     @OnStopped
+     public void stopped() {
+         sessionFactoryRef.set(null);
+     }
+ 
+     public static class HostValidator implements Validator {
+ 
+         @Override
+         public ValidationResult validate(String subject, String input, ValidationContext context) {
+             try {
+                 InetAddress.getByName(input);
+                 return new ValidationResult.Builder()
+                         .subject(subject)
+                         .valid(true)
+                         .input(input)
+                         .build();
+             } catch (final UnknownHostException e) {
+                 return new ValidationResult.Builder()
+                         .subject(subject)
+                         .valid(false)
+                         .input(input)
+                         .explanation("Unknown host: " + e)
+                         .build();
+             }
+         }
+ 
+     }
+ 
+ }


[6/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 0000000,3d3e854..dcb461c
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@@ -1,0 -1,2689 +1,2684 @@@
+ /*
+  * 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.controller.repository;
+ 
+ import java.io.ByteArrayInputStream;
+ import java.io.EOFException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Files;
+ import java.nio.file.Path;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.LinkedHashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.NoSuchElementException;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.regex.Pattern;
+ 
+ import org.apache.nifi.connectable.Connectable;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.FlowFileQueue;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.repository.claim.ContentClaim;
+ import org.apache.nifi.controller.repository.io.ByteCountingInputStream;
+ import org.apache.nifi.controller.repository.io.ByteCountingOutputStream;
+ import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream;
+ import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream;
+ import org.apache.nifi.controller.repository.io.FlowFileAccessOutputStream;
+ import org.apache.nifi.controller.repository.io.LimitedInputStream;
+ import org.apache.nifi.controller.repository.io.LongHolder;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedOutputStream;
+ import org.apache.nifi.stream.io.NonCloseableInputStream;
+ import org.apache.nifi.stream.io.StreamUtils;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.MissingFlowFileException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceEventBuilder;
+ import org.apache.nifi.provenance.ProvenanceEventRecord;
+ import org.apache.nifi.provenance.ProvenanceEventRepository;
+ import org.apache.nifi.provenance.ProvenanceEventType;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+ import org.apache.nifi.util.NiFiProperties;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /**
+  * <p>
+  * Provides a ProcessSession that ensures all accesses, changes and transfers
+  * occur in an atomic manner for all FlowFiles including their contents and
+  * attributes</p>
+  * <p>
+  * NOT THREAD SAFE</p>
+  * <p/>
+  * @author none
+  */
+ public final class StandardProcessSession implements ProcessSession, ProvenanceEventEnricher {
+ 
+     private static final AtomicLong idGenerator = new AtomicLong(0L);
+ 
+     // determines how many things must be transferred, removed, modified in order to avoid logging the FlowFile ID's on commit/rollback
+     public static final int VERBOSE_LOG_THRESHOLD = 10;
+     private static final long MAX_APPENDABLE_CLAIM_SIZE = DataUnit.parseDataSize(
+             NiFiProperties.getInstance().getMaxAppendableClaimSize(), DataUnit.B).longValue();
+     private static final int MAX_FLOWFILES_PER_CLAIM = NiFiProperties.getInstance().getMaxFlowFilesPerClaim();
+ 
+     public static final String DEFAULT_FLOWFILE_PATH = "./";
+ 
+     private static final Logger LOG = LoggerFactory.getLogger(StandardProcessSession.class);
+     private static final Logger claimLog = LoggerFactory.getLogger(StandardProcessSession.class.getSimpleName() + ".claims");
+ 
+     private final Map<FlowFileRecord, StandardRepositoryRecord> records = new HashMap<>();
+     private final Map<Connection, StandardFlowFileEvent> connectionCounts = new HashMap<>();
+     private final Map<Connection, Set<FlowFileRecord>> unacknowledgedFlowFiles = new HashMap<>();
+     private final Map<String, Long> localCounters = new HashMap<>();
+     private final Map<String, Long> globalCounters = new HashMap<>();
+     private final Map<ContentClaim, ByteCountingOutputStream> appendableStreams = new HashMap<>();
+     private final ProcessContext context;
+     private final Set<FlowFile> recursionSet = new HashSet<>();//set used to track what is currently being operated on to prevent logic failures if recursive calls occurring
+     private final Set<Path> deleteOnCommit = new HashSet<>();
+     private final long sessionId;
+     private final String connectableDescription;
+ 
+     private final Set<String> removedFlowFiles = new HashSet<>();
+     private final Set<String> createdFlowFiles = new HashSet<>();
+ 
+     private final StandardProvenanceReporter provenanceReporter;
+ 
+     private int removedCount = 0;    // number of flowfiles removed in this session
+     private long removedBytes = 0L; // size of all flowfiles removed in this session
+     private LongHolder bytesRead = new LongHolder(0L);
+     private LongHolder bytesWritten = new LongHolder(0L);
+     private int flowFilesIn = 0, flowFilesOut = 0;
+     private long contentSizeIn = 0L, contentSizeOut = 0L;
+     private int writeRecursionLevel = 0;
+ 
+     private ContentClaim currentWriteClaim = null;
+     private OutputStream currentWriteClaimStream = null;
+     private long currentWriteClaimSize = 0L;
+     private int currentWriteClaimFlowFileCount = 0;
+ 
+     private ContentClaim currentReadClaim = null;
+     private ByteCountingInputStream currentReadClaimStream = null;
+     private long processingStartTime;
+ 
+     // maps a FlowFile to all Provenance Events that were generated for that FlowFile.
+     // we do this so that if we generate a Fork event, for example, and then remove the event in the same
+     // Session, we will not send that event to the Provenance Repository
+     private Map<FlowFile, List<ProvenanceEventRecord>> generatedProvenanceEvents = new HashMap<>();
+ 
+     // when Forks are generated for a single parent, we add the Fork event to this map, with the Key being the parent
+     // so that we are able to aggregate many into a single Fork Event.
+     private Map<FlowFile, ProvenanceEventBuilder> forkEventBuilders = new HashMap<>();
+ 
+     private Checkpoint checkpoint = new Checkpoint();
+ 
+     public StandardProcessSession(final ProcessContext context) {
+         this.context = context;
+ 
+         final Connectable connectable = context.getConnectable();
+         final String componentType;
+ 
+         String description = connectable.toString();
+         switch (connectable.getConnectableType()) {
+             case PROCESSOR:
+                 final ProcessorNode procNode = (ProcessorNode) connectable;
+                 componentType = procNode.getProcessor().getClass().getSimpleName();
+                 description = procNode.getProcessor().toString();
+                 break;
+             case INPUT_PORT:
+                 componentType = "Input Port";
+                 break;
+             case OUTPUT_PORT:
+                 componentType = "Output Port";
+                 break;
+             case REMOTE_INPUT_PORT:
+                 componentType = "Remote Input Port";
+                 break;
+             case REMOTE_OUTPUT_PORT:
+                 componentType = "Remote Output Port";
+                 break;
+             case FUNNEL:
+                 componentType = "Funnel";
+                 break;
+             default:
+                 throw new AssertionError("Connectable type is " + connectable.getConnectableType());
+         }
+ 
+         this.provenanceReporter = new StandardProvenanceReporter(connectable.getIdentifier(), componentType, context.getProvenanceRepository(), this);
+         this.sessionId = idGenerator.getAndIncrement();
+         this.connectableDescription = description;
+ 
+         LOG.trace("Session {} created for {}", this, connectableDescription);
+         processingStartTime = System.nanoTime();
+     }
+ 
+     public void checkpoint() {
+         if (!recursionSet.isEmpty()) {
+             throw new IllegalStateException();
+         }
+ 
+         if (this.checkpoint == null) {
+             this.checkpoint = new Checkpoint();
+         }
+ 
+         if (records.isEmpty()) {
+             LOG.trace("{} checkpointed, but no events were performed by this ProcessSession", this);
+             return;
+         }
+ 
+         // any drop event that is the result of an auto-terminate should happen at the very end, so we keep the
+         // records in a separate List so that they can be persisted to the Provenance Repo after all of the
+         // Processor-reported events.
+         List<ProvenanceEventRecord> autoTerminatedEvents = null;
+ 
+         //validate that all records have a transfer relationship for them and if so determine the destination node and clone as necessary
+         final Map<FlowFileRecord, StandardRepositoryRecord> toAdd = new HashMap<>();
+         for (final StandardRepositoryRecord record : records.values()) {
+             if (record.isMarkedForDelete()) {
+                 continue;
+             }
+             final Relationship relationship = record.getTransferRelationship();
+             if (relationship == null) {
+                 rollback();
+                 throw new FlowFileHandlingException(record.getCurrent() + " transfer relationship not specified");
+             }
+             final List<Connection> destinations = new ArrayList<>(context.getConnections(relationship));
+             if (destinations.isEmpty() && !context.getConnectable().isAutoTerminated(relationship)) {
+                 if (relationship != Relationship.SELF) {
+                     rollback();
+                     throw new FlowFileHandlingException(relationship + " does not have any destinations for " + context.getConnectable());
+                 }
+             }
+ 
+             if (destinations.isEmpty() && relationship == Relationship.SELF) {
+                 record.setDestination(record.getOriginalQueue());
+             } else if (destinations.isEmpty()) {
+                 record.markForDelete();
+ 
+                 if (autoTerminatedEvents == null) {
+                     autoTerminatedEvents = new ArrayList<>();
+                 }
+ 
+                 final ProvenanceEventRecord dropEvent;
+                 try {
+                     dropEvent = provenanceReporter.generateDropEvent(record.getCurrent(), "Auto-Terminated by " + relationship.getName() + " Relationship");
+                     autoTerminatedEvents.add(dropEvent);
+                 } catch (final Exception e) {
+                     LOG.warn("Unable to generate Provenance Event for {} on behalf of {} due to {}", record.getCurrent(), connectableDescription, e);
+                     if (LOG.isDebugEnabled()) {
+                         LOG.warn("", e);
+                     }
+                 }
+             } else {
+                 final Connection finalDestination = destinations.remove(destinations.size() - 1); //remove last element
+                 record.setDestination(finalDestination.getFlowFileQueue());
+                 incrementConnectionInputCounts(finalDestination, record);
+ 
+                 for (final Connection destination : destinations) { //iterate over remaining destinations and "clone" as needed
+                     incrementConnectionInputCounts(destination, record);
+                     final FlowFileRecord currRec = record.getCurrent();
+                     final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec);
+                     builder.id(context.getNextFlowFileSequence());
+ 
+                     final String newUuid = UUID.randomUUID().toString();
+                     builder.addAttribute(CoreAttributes.UUID.key(), newUuid);
+ 
+                     final FlowFileRecord clone = builder.build();
+                     final StandardRepositoryRecord newRecord = new StandardRepositoryRecord(destination.getFlowFileQueue());
+                     getProvenanceReporter().clone(currRec, clone);
+ 
+                     final ContentClaim claim = clone.getContentClaim();
+                     if (claim != null) {
+                         context.getContentRepository().incrementClaimaintCount(claim);
+                     }
+                     newRecord.setWorking(clone, CoreAttributes.UUID.key(), newUuid);
+ 
+                     newRecord.setDestination(destination.getFlowFileQueue());
+                     newRecord.setTransferRelationship(record.getTransferRelationship());
+                     // put the mapping into toAdd because adding to records now will cause a ConcurrentModificationException
+                     toAdd.put(clone, newRecord);
+                 }
+             }
+         }
+ 
+         records.putAll(toAdd);
+         toAdd.clear();
+ 
+         checkpoint.checkpoint(this, autoTerminatedEvents);
+         resetState();
+     }
+ 
+     @Override
+     public void commit() {
+         checkpoint();
+         commit(this.checkpoint);
+         this.checkpoint = null;
+     }
+ 
+     @SuppressWarnings({"unchecked", "rawtypes"})
+     private void commit(final Checkpoint checkpoint) {
+         final long commitStartNanos = System.nanoTime();
+ 
+         resetWriteClaims();
+         resetReadClaim();
+ 
+         final long updateProvenanceStart = System.nanoTime();
+         updateProvenanceRepo(checkpoint);
+ 
+         final long claimRemovalStart = System.nanoTime();
+         final long updateProvenanceNanos = claimRemovalStart - updateProvenanceStart;
+ 
+         // Figure out which content claims can be released.
+         // At this point, we will decrement the Claimant Count for the claims via the Content Repository.
+         // We do not actually destroy the content because otherwise, we could remove the 
+         // Original Claim and crash/restart before the FlowFileRepository is updated. This will result in the FlowFile being restored such that
+         // the content claim points to the Original Claim -- which has already been removed!
+         for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+             final FlowFile flowFile = entry.getKey();
+             final StandardRepositoryRecord record = entry.getValue();
+ 
+             if (record.isMarkedForDelete()) {
+                 // if the working claim is not the same as the original claim, we can immediately destroy the working claim
+                 // because it was created in this session and is to be deleted. We don't need to wait for the FlowFile Repo to sync.
+                 removeContent(record.getWorkingClaim());
+ 
+                 if (record.getOriginalClaim() != null && !record.getOriginalClaim().equals(record.getWorkingClaim())) {
+                     // if working & original claim are same, don't remove twice; we only want to remove the original
+                     // if it's different from the working. Otherwise, we remove two claimant counts. This causes
+                     // an issue if we only updated the FlowFile attributes.
+                     removeContent(record.getOriginalClaim());
+                 }
+                 final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
+                 final Connectable connectable = context.getConnectable();
+                 final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
+                 LOG.info("{} terminated by {}; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
+             } else if (record.isWorking() && record.getWorkingClaim() != record.getOriginalClaim()) {
+                 //records which have been updated - remove original if exists
+                 removeContent(record.getOriginalClaim());
+             }
+         }
+ 
+         final long claimRemovalFinishNanos = System.nanoTime();
+         final long claimRemovalNanos = claimRemovalFinishNanos - claimRemovalStart;
+ 
+         // Update the FlowFile Repository
+         try {
+             final Collection<StandardRepositoryRecord> repoRecords = checkpoint.records.values();
+             context.getFlowFileRepository().updateRepository((Collection) repoRecords);
+         } catch (final IOException ioe) {
+             rollback();
+             throw new ProcessException("FlowFile Repository failed to update", ioe);
+         }
+         final long flowFileRepoUpdateFinishNanos = System.nanoTime();
+         final long flowFileRepoUpdateNanos = flowFileRepoUpdateFinishNanos - claimRemovalFinishNanos;
+ 
+         updateEventRepository(checkpoint);
+ 
+         final long updateEventRepositoryFinishNanos = System.nanoTime();
+         final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - claimRemovalFinishNanos;
+ 
+         // transfer the flowfiles to the connections' queues.
+         final Map<FlowFileQueue, Collection<FlowFileRecord>> recordMap = new HashMap<>();
+         for (final StandardRepositoryRecord record : checkpoint.records.values()) {
+             if (record.isMarkedForAbort() || record.isMarkedForDelete()) {
+                 continue; //these don't need to be transferred
+             }
+             // record.getCurrent() will return null if this record was created in this session --
+             // in this case, we just ignore it, and it will be cleaned up by clearing the records map.
+             if (record.getCurrent() != null) {
+                 Collection<FlowFileRecord> collection = recordMap.get(record.getDestination());
+                 if (collection == null) {
+                     collection = new ArrayList<>();
+                     recordMap.put(record.getDestination(), collection);
+                 }
+                 collection.add(record.getCurrent());
+             }
+         }
+ 
+         for (final Map.Entry<FlowFileQueue, Collection<FlowFileRecord>> entry : recordMap.entrySet()) {
+             entry.getKey().putAll(entry.getValue());
+         }
+ 
+         final long enqueueFlowFileFinishNanos = System.nanoTime();
+         final long enqueueFlowFileNanos = enqueueFlowFileFinishNanos - updateEventRepositoryFinishNanos;
+ 
+         // Delete any files from disk that need to be removed.
+         for (final Path path : checkpoint.deleteOnCommit) {
+             try {
+                 Files.deleteIfExists(path);
+             } catch (final IOException e) {
+                 throw new FlowFileAccessException("Unable to delete " + path.toFile().getAbsolutePath(), e);
+             }
+         }
+         checkpoint.deleteOnCommit.clear();
+ 
+         if (LOG.isInfoEnabled()) {
+             final String sessionSummary = summarizeEvents(checkpoint);
+             if (!sessionSummary.isEmpty()) {
+                 LOG.info("{} for {}, committed the following events: {}", new Object[]{this, connectableDescription, sessionSummary});
+             }
+         }
+ 
+         for (final Map.Entry<String, Long> entry : checkpoint.localCounters.entrySet()) {
+             adjustCounter(entry.getKey(), entry.getValue(), true);
+         }
+ 
+         for (final Map.Entry<String, Long> entry : checkpoint.globalCounters.entrySet()) {
+             adjustCounter(entry.getKey(), entry.getValue(), true);
+         }
+ 
+         acknowledgeRecords();
+         resetState();
+ 
+         if (LOG.isDebugEnabled()) {
+             final StringBuilder timingInfo = new StringBuilder();
+             timingInfo.append("Session commit for ").append(this).append(" [").append(connectableDescription).append("]").append(" took ");
+ 
+             final long commitNanos = System.nanoTime() - commitStartNanos;
+             formatNanos(commitNanos, timingInfo);
+             timingInfo.append("; FlowFile Repository Update took ");
+             formatNanos(flowFileRepoUpdateNanos, timingInfo);
+             timingInfo.append("; Claim Removal took ");
+             formatNanos(claimRemovalNanos, timingInfo);
+             timingInfo.append("; FlowFile Event Update took ");
+             formatNanos(updateEventRepositoryNanos, timingInfo);
+             timingInfo.append("; Enqueuing FlowFiles took ");
+             formatNanos(enqueueFlowFileNanos, timingInfo);
+             timingInfo.append("; Updating Provenance Event Repository took ");
+             formatNanos(updateProvenanceNanos, timingInfo);
+ 
+             LOG.debug(timingInfo.toString());
+         }
+     }
+ 
+     private void updateEventRepository(final Checkpoint checkpoint) {
+         int flowFilesReceived = 0;
+         int flowFilesSent = 0;
+         long bytesReceived = 0L;
+         long bytesSent = 0L;
+ 
+         for (final ProvenanceEventRecord event : checkpoint.reportedEvents) {
+             if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+                 continue;
+             }
+ 
+             switch (event.getEventType()) {
+                 case SEND:
+                     flowFilesSent++;
+                     bytesSent += event.getFileSize();
+                     break;
+                 case RECEIVE:
+                     flowFilesReceived++;
+                     bytesReceived += event.getFileSize();
+                     break;
+                 default:
+                     break;
+             }
+         }
+ 
+         try {
+             // update event repository
+             final Connectable connectable = context.getConnectable();
+             final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier());
+             flowFileEvent.setBytesRead(checkpoint.bytesRead);
+             flowFileEvent.setBytesWritten(checkpoint.bytesWritten);
+             flowFileEvent.setContentSizeIn(checkpoint.contentSizeIn);
+             flowFileEvent.setContentSizeOut(checkpoint.contentSizeOut);
+             flowFileEvent.setContentSizeRemoved(checkpoint.removedBytes);
+             flowFileEvent.setFlowFilesIn(checkpoint.flowFilesIn);
+             flowFileEvent.setFlowFilesOut(checkpoint.flowFilesOut);
+             flowFileEvent.setFlowFilesRemoved(checkpoint.removedCount);
+             flowFileEvent.setFlowFilesReceived(flowFilesReceived);
+             flowFileEvent.setBytesReceived(bytesReceived);
+             flowFileEvent.setFlowFilesSent(flowFilesSent);
+             flowFileEvent.setBytesSent(bytesSent);
+ 
+             long lineageMillis = 0L;
+             for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+                 final FlowFile flowFile = entry.getKey();
+                 final long lineageDuration = System.currentTimeMillis() - flowFile.getLineageStartDate();
+                 lineageMillis += lineageDuration;
+             }
+             flowFileEvent.setAggregateLineageMillis(lineageMillis);
+ 
+             context.getFlowFileEventRepository().updateRepository(flowFileEvent);
+ 
+             for (final FlowFileEvent connectionEvent : checkpoint.connectionCounts.values()) {
+                 context.getFlowFileEventRepository().updateRepository(connectionEvent);
+             }
+         } catch (final IOException ioe) {
+             LOG.error("FlowFile Event Repository failed to update", ioe);
+         }
+     }
+ 
+     private void addEventType(final Map<String, Set<ProvenanceEventType>> map, final String id, final ProvenanceEventType eventType) {
+         Set<ProvenanceEventType> eventTypes = map.get(id);
+         if ( eventTypes == null ) {
+             eventTypes = new HashSet<>();
+             map.put(id, eventTypes);
+         }
+         
+         eventTypes.add(eventType);
+     }
+     
+     private void updateProvenanceRepo(final Checkpoint checkpoint) {
+         // Update Provenance Repository
+         final ProvenanceEventRepository provenanceRepo = context.getProvenanceRepository();
+ 
+         // We need to de-dupe the events that we've created and those reported to the provenance reporter,
+         // in case the Processor developer submitted the same events to the reporter. So we use a LinkedHashSet
+         // for this, so that we are able to ensure that the events are submitted in the proper order.
+         final Set<ProvenanceEventRecord> recordsToSubmit = new LinkedHashSet<>();
+         final Map<String, Set<ProvenanceEventType>> eventTypesPerFlowFileId = new HashMap<>();
+         
+         final Set<ProvenanceEventRecord> processorGenerated = checkpoint.reportedEvents;
+ 
+         // We first want to submit FORK events because if the Processor is going to create events against
+         // a FlowFile, that FlowFile needs to be shown to be created first.
+         // However, if the Processor has generated a FORK event, we don't want to use the Framework-created one --
+         // we prefer to use the event generated by the Processor. We can determine this by checking if the Set of events genereated
+         // by the Processor contains any of the FORK events that we generated
+         for (final Map.Entry<FlowFile, ProvenanceEventBuilder> entry : checkpoint.forkEventBuilders.entrySet()) {
+             final ProvenanceEventBuilder builder = entry.getValue();
+             final FlowFile flowFile = entry.getKey();
+ 
+             updateEventContentClaims(builder, flowFile, checkpoint.records.get(flowFile));
+             final ProvenanceEventRecord event = builder.build();
+ 
+             if (!event.getChildUuids().isEmpty() && !isSpuriousForkEvent(event, checkpoint.removedFlowFiles) && !processorGenerated.contains(event)) {
+                 recordsToSubmit.add(event);
+                 
+                 for ( final String childUuid : event.getChildUuids() ) {
+                     addEventType(eventTypesPerFlowFileId, childUuid, event.getEventType());
+                 }
+                 for ( final String parentUuid : event.getParentUuids() ) {
+                     addEventType(eventTypesPerFlowFileId, parentUuid, event.getEventType());
+                 }
+             }
+         }
+ 
+         // Now add any Processor-reported events.
+         for (final ProvenanceEventRecord event : processorGenerated) {
+             if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+                 continue;
+             }
+             if ( isSpuriousRouteEvent(event, checkpoint.records) ) {
+                 continue;
+             }
+             
+             // Check if the event indicates that the FlowFile was routed to the same 
+             // connection from which it was pulled (and only this connection). If so, discard the event.
+             isSpuriousRouteEvent(event, checkpoint.records);
+             
+             recordsToSubmit.add(event);
+             addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType());
+         }
+ 
+         // Finally, add any other events that we may have generated.
+         for (final List<ProvenanceEventRecord> eventList : checkpoint.generatedProvenanceEvents.values()) {
+             for (final ProvenanceEventRecord event : eventList) {
+                 if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+                     continue;
+                 }
+ 
+                 recordsToSubmit.add(event);
+                 addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType());
+             }
+         }
+         
+         // Check if content or attributes changed. If so, register the appropriate events.
+         for (final StandardRepositoryRecord repoRecord : checkpoint.records.values() ) {
+             final ContentClaim original = repoRecord.getOriginalClaim();
+             final ContentClaim current = repoRecord.getCurrentClaim();
+             
+             boolean contentChanged = false;
+             if ( original == null && current != null ) {
+                 contentChanged = true;
+             }
+             if ( original != null && current == null ) {
+                 contentChanged = true;
+             }
+             if ( original != null && current != null && !original.equals(current) ) {
+                 contentChanged = true;
+             }
+             
+             final FlowFileRecord curFlowFile = repoRecord.getCurrent();
+             final String flowFileId = curFlowFile.getAttribute(CoreAttributes.UUID.key());
+             boolean eventAdded = false;
+             
+             if (checkpoint.removedFlowFiles.contains(flowFileId)) {
+                 continue;
+             }
+             
+             final boolean newFlowFile = repoRecord.getOriginal() == null;
+             if ( contentChanged && !newFlowFile ) {
+                 recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.CONTENT_MODIFIED).build());
+                 addEventType(eventTypesPerFlowFileId, flowFileId, ProvenanceEventType.CONTENT_MODIFIED);
+                 eventAdded = true;
+             }
+             
+             if ( checkpoint.createdFlowFiles.contains(flowFileId) ) {
+                 final Set<ProvenanceEventType> registeredTypes = eventTypesPerFlowFileId.get(flowFileId);
+                 boolean creationEventRegistered = false;
+                 if ( registeredTypes != null ) {
+                     if ( registeredTypes.contains(ProvenanceEventType.CREATE) ||
+                             registeredTypes.contains(ProvenanceEventType.FORK) ||
+                             registeredTypes.contains(ProvenanceEventType.JOIN) ||
+                             registeredTypes.contains(ProvenanceEventType.RECEIVE) ) {
+                         creationEventRegistered = true;
+                     }
+                 }
+                 
+                 if ( !creationEventRegistered ) {
+                     recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.CREATE).build());
+                     eventAdded = true;
+                 }
+             }
+             
+             if ( !eventAdded && !repoRecord.getUpdatedAttributes().isEmpty() ) {
+                 // We generate an ATTRIBUTES_MODIFIED event only if no other event has been
+                 // created for the FlowFile. We do this because all events contain both the
+                 // newest and the original attributes, so generating an ATTRIBUTES_MODIFIED
+                 // event is redundant if another already exists.
+                 if ( !eventTypesPerFlowFileId.containsKey(flowFileId) ) {
+                     recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.ATTRIBUTES_MODIFIED).build());
+                     addEventType(eventTypesPerFlowFileId, flowFileId, ProvenanceEventType.ATTRIBUTES_MODIFIED);
+                 }
+             }
+         }
+ 
+         // We want to submit the 'recordsToSubmit' collection, followed by the auto-terminated events to the Provenance Repository.
+         // We want to do this with a single call to ProvenanceEventRepository#registerEvents because it may be much more efficient
+         // to do so.
+         // However, we want to modify the events in 'recordsToSubmit' to obtain the data from the most recent version of the FlowFiles
+         // (except for SEND events); see note below as to why this is
+         // Therefore, we create an Iterable that can iterate over each of these events, modifying them as needed, and returning them
+         // in the appropriate order. This prevents an unnecessary step of creating an intermediate List and adding all of those values
+         // to the List.
+         // This is done in a similar veign to how Java 8's streams work, iterating over the events and returning a processed version
+         // one-at-a-time as opposed to iterating over the entire Collection and putting the results in another Collection. However,
+         // we don't want to change the Framework to require Java 8 at this time, because it's not yet as prevalent as we would desire
+         final Map<String, FlowFileRecord> flowFileRecordMap = new HashMap<>();
+         for (final StandardRepositoryRecord repoRecord : checkpoint.records.values()) {
+             final FlowFileRecord flowFile = repoRecord.getCurrent();
+             flowFileRecordMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+         }
+ 
+         final List<ProvenanceEventRecord> autoTermEvents = checkpoint.autoTerminatedEvents;
+         final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
+             final Iterator<ProvenanceEventRecord> recordsToSubmitIterator = recordsToSubmit.iterator();
+             final Iterator<ProvenanceEventRecord> autoTermIterator = autoTermEvents == null ? null : autoTermEvents.iterator();
+ 
+             @Override
+             public Iterator<ProvenanceEventRecord> iterator() {
+                 return new Iterator<ProvenanceEventRecord>() {
+                     @Override
+                     public boolean hasNext() {
+                         return recordsToSubmitIterator.hasNext() || (autoTermIterator != null && autoTermIterator.hasNext());
+                     }
+ 
+                     @Override
+                     public ProvenanceEventRecord next() {
+                         if (recordsToSubmitIterator.hasNext()) {
+                             final ProvenanceEventRecord rawEvent = recordsToSubmitIterator.next();
+ 
+                             // Update the Provenance Event Record with all of the info that we know about the event.
+                             // For SEND events, we do not want to update the FlowFile info on the Event, because the event should
+                             // reflect the FlowFile as it was sent to the remote system. However, for other events, we want to use
+                             // the representation of the FlowFile as it is committed, as this is the only way in which it really
+                             // exists in our system -- all other representations are volatile representations that have not been
+                             // exposed.
+                             return enrich(rawEvent, flowFileRecordMap, checkpoint.records, rawEvent.getEventType() != ProvenanceEventType.SEND);
+                         } else if (autoTermIterator != null && autoTermIterator.hasNext()) {
+                             return enrich(autoTermIterator.next(), flowFileRecordMap, checkpoint.records, true);
+                         }
+ 
+                         throw new NoSuchElementException();
+                     }
+ 
+                     @Override
+                     public void remove() {
+                         throw new UnsupportedOperationException();
+                     }
+                 };
+             }
+         };
+ 
+         provenanceRepo.registerEvents(iterable);
+     }
+ 
+     private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) {
+         final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+         if (originalClaim == null) {
+             builder.setCurrentContentClaim(null, null, null, null, 0L);
+         } else {
+             builder.setCurrentContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), repoRecord.getOriginal().getContentClaimOffset(), repoRecord.getOriginal().getSize());
+             builder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), repoRecord.getOriginal().getContentClaimOffset(), repoRecord.getOriginal().getSize());
+         }
+     }
+ 
+     @Override
+     public StandardProvenanceEventRecord enrich(final ProvenanceEventRecord rawEvent, final FlowFile flowFile) {
+         final StandardRepositoryRecord repoRecord = records.get(flowFile);
+         if (repoRecord == null) {
+             throw new FlowFileHandlingException(flowFile + " is not known in this session (" + toString() + ")");
+         }
+ 
+         final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder().fromEvent(rawEvent);
+         if (repoRecord.getCurrent() != null && repoRecord.getCurrentClaim() != null) {
+             final ContentClaim currentClaim = repoRecord.getCurrentClaim();
+             final long currentOffset = repoRecord.getCurrentClaimOffset();
+             final long size = flowFile.getSize();
+             recordBuilder.setCurrentContentClaim(currentClaim.getContainer(), currentClaim.getSection(), currentClaim.getId(), currentOffset, size);
+         }
+ 
+         if (repoRecord.getOriginal() != null && repoRecord.getOriginalClaim() != null) {
+             final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+             final long originalOffset = repoRecord.getOriginal().getContentClaimOffset();
+             final long originalSize = repoRecord.getOriginal().getSize();
+             recordBuilder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), originalOffset, originalSize);
+         }
+ 
+         final FlowFileQueue originalQueue = repoRecord.getOriginalQueue();
+         if (originalQueue != null) {
+             recordBuilder.setSourceQueueIdentifier(originalQueue.getIdentifier());
+         }
+ 
+         recordBuilder.setAttributes(repoRecord.getOriginalAttributes(), repoRecord.getUpdatedAttributes());
+         return recordBuilder.build();
+     }
+ 
+     private StandardProvenanceEventRecord enrich(final ProvenanceEventRecord rawEvent, final Map<String, FlowFileRecord> flowFileRecordMap, final Map<FlowFileRecord, StandardRepositoryRecord> records, final boolean updateAttributes) {
+         final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder().fromEvent(rawEvent);
+         final FlowFileRecord eventFlowFile = flowFileRecordMap.get(rawEvent.getFlowFileUuid());
+         if (eventFlowFile != null) {
+             final StandardRepositoryRecord repoRecord = records.get(eventFlowFile);
+ 
+             if (repoRecord.getCurrent() != null && repoRecord.getCurrentClaim() != null) {
+                 final ContentClaim currentClaim = repoRecord.getCurrentClaim();
+                 final long currentOffset = repoRecord.getCurrentClaimOffset();
+                 final long size = eventFlowFile.getSize();
+                 recordBuilder.setCurrentContentClaim(currentClaim.getContainer(), currentClaim.getSection(), currentClaim.getId(), currentOffset, size);
+             }
+ 
+             if (repoRecord.getOriginal() != null && repoRecord.getOriginalClaim() != null) {
+                 final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+                 final long originalOffset = repoRecord.getOriginal().getContentClaimOffset();
+                 final long originalSize = repoRecord.getOriginal().getSize();
+                 recordBuilder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), originalOffset, originalSize);
+             }
+ 
+             final FlowFileQueue originalQueue = repoRecord.getOriginalQueue();
+             if (originalQueue != null) {
+                 recordBuilder.setSourceQueueIdentifier(originalQueue.getIdentifier());
+             }
+         }
+ 
+         if (updateAttributes) {
+             final FlowFileRecord flowFileRecord = flowFileRecordMap.get(rawEvent.getFlowFileUuid());
+             if (flowFileRecord != null) {
+                 final StandardRepositoryRecord record = records.get(flowFileRecord);
+                 if (record != null) {
+                     recordBuilder.setAttributes(record.getOriginalAttributes(), record.getUpdatedAttributes());
+                 }
+             }
+         }
+ 
+         return recordBuilder.build();
+     }
+ 
+     /**
+      * Checks if the given event is a spurious FORK, meaning that the FORK has a
+      * single child and that child was removed in this session. This happens
+      * when a Processor calls #create(FlowFile) and then removes the created
+      * FlowFile.
+      *
+      * @param event
+      * @return
+      */
+     private boolean isSpuriousForkEvent(final ProvenanceEventRecord event, final Set<String> removedFlowFiles) {
+         if (event.getEventType() == ProvenanceEventType.FORK) {
+             final List<String> childUuids = event.getChildUuids();
+             if (childUuids != null && childUuids.size() == 1 && removedFlowFiles.contains(childUuids.get(0))) {
+                 return true;
+             }
+         }
+ 
+         return false;
+     }
+ 
+     
+     /**
+      * Checks if the given event is a spurious ROUTE, meaning that the ROUTE indicates that a FlowFile
+      * was routed to a relationship with only 1 connection and that Connection is the Connection from which
+      * the FlowFile was pulled. I.e., the FlowFile was really routed nowhere.
+      * 
+      * @param event
+      * @param records
+      * @return
+      */
+     private boolean isSpuriousRouteEvent(final ProvenanceEventRecord event, final Map<FlowFileRecord, StandardRepositoryRecord> records) {
+         if ( event.getEventType() == ProvenanceEventType.ROUTE ) {
+             final String relationshipName = event.getRelationship();
+             final Relationship relationship = new Relationship.Builder().name(relationshipName).build();
+             final Collection<Connection> connectionsForRelationship = this.context.getConnections(relationship);
+             
+             // If the number of connections for this relationship is not 1, then we can't ignore this ROUTE event,
+             // as it may be cloning the FlowFile and adding to multiple connections.
+             if ( connectionsForRelationship.size() == 1 ) {
+                 for ( final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : records.entrySet() ) {
+                     final FlowFileRecord flowFileRecord = entry.getKey();
+                     if ( event.getFlowFileUuid().equals(flowFileRecord.getAttribute(CoreAttributes.UUID.key())) ) {
+                         final StandardRepositoryRecord repoRecord = entry.getValue();
+                         if ( repoRecord.getOriginalQueue() == null ) {
+                             return false;
+                         }
+                         
+                         final String originalQueueId = repoRecord.getOriginalQueue().getIdentifier();
+                         final Connection destinationConnection = connectionsForRelationship.iterator().next();
+                         final String destinationQueueId = destinationConnection.getFlowFileQueue().getIdentifier();
+                         return originalQueueId.equals(destinationQueueId);
+                     }
+                 }
+             }
+         }
+         
+         return false;
+     }
+     
+     @Override
+     public void rollback() {
+         rollback(false);
+     }
+ 
+     @Override
+     public void rollback(final boolean penalize) {
+         deleteOnCommit.clear();
+         if (records.isEmpty()) {
+             LOG.trace("{} was rolled back, but no events were performed by this ProcessSession", this);
+             acknowledgeRecords();
+             return;
+         }
+ 
+         resetWriteClaims();
+         resetReadClaim();
+ 
+         for (final StandardRepositoryRecord record : records.values()) {
+             // remove the working claim if it's different than the original.
+             removeTemporaryClaim(record);
+         }
+ 
+         final Set<RepositoryRecord> abortedRecords = new HashSet<>();
+         final Set<StandardRepositoryRecord> transferRecords = new HashSet<>();
+         for (final StandardRepositoryRecord record : records.values()) {
+             if (record.isMarkedForAbort()) {
+                 removeContent(record.getWorkingClaim());
+                 if (record.getCurrentClaim() != null && !record.getCurrentClaim().equals(record.getWorkingClaim())) {
+                     // if working & original claim are same, don't remove twice; we only want to remove the original
+                     // if it's different from the working. Otherwise, we remove two claimant counts. This causes
+                     // an issue if we only updated the flowfile attributes.
+                     removeContent(record.getCurrentClaim());
+                 }
+                 abortedRecords.add(record);
+             } else {
+                 transferRecords.add(record);
+             }
+         }
+ 
+         // Put the FlowFiles that are not marked for abort back to their original queues
+         for (final StandardRepositoryRecord record : transferRecords) {
+             if (record.getOriginal() != null) {
+                 final FlowFileQueue originalQueue = record.getOriginalQueue();
+                 if (originalQueue != null) {
+                     if (penalize) {
+                         final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS);
+                         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getOriginal()).penaltyExpirationTime(expirationEpochMillis).build();
+                         originalQueue.put(newFile);
+                     } else {
+                         originalQueue.put(record.getOriginal());
+                     }
+                 }
+             }
+         }
+ 
+         if (!abortedRecords.isEmpty()) {
+             try {
+                 context.getFlowFileRepository().updateRepository(abortedRecords);
+             } catch (final IOException ioe) {
+                 LOG.error("Unable to update FlowFile repository for aborted records due to {}", ioe.toString());
+                 if (LOG.isDebugEnabled()) {
+                     LOG.error("", ioe);
+                 }
+             }
+         }
+ 
+         final Connectable connectable = context.getConnectable();
+         final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier());
+         flowFileEvent.setBytesRead(bytesRead.getValue());
+         flowFileEvent.setBytesWritten(bytesWritten.getValue());
+ 
+         // update event repository
+         try {
+             context.getFlowFileEventRepository().updateRepository(flowFileEvent);
+         } catch (final Exception e) {
+             LOG.error("Failed to update FlowFileEvent Repository due to " + e);
+             if (LOG.isDebugEnabled()) {
+                 LOG.error("", e);
+             }
+         }
+ 
+         acknowledgeRecords();
+         resetState();
+     }
+ 
+     private void removeContent(final ContentClaim claim) {
+         if (claim == null) {
+             return;
+         }
+ 
+         context.getContentRepository().decrementClaimantCount(claim);
+     }
+ 
+     /**
+      * Destroys a ContentClaim that was being written to but is no longer needed
+      *
+      * @param claim
+      */
+     private void destroyContent(final ContentClaim claim) {
+         if (claim == null) {
+             return;
+         }
+ 
+         final int decrementedClaimCount = context.getContentRepository().decrementClaimantCount(claim);
+         if (decrementedClaimCount <= 0) {
+             resetWriteClaims(); // Have to ensure that we are not currently writing to the claim before we can destroy it.
+             context.getContentRepository().remove(claim);
+         }
+     }
+ 
+     private void resetState() {
+         records.clear();
+         recursionSet.clear();
+         contentSizeIn = 0L;
+         contentSizeOut = 0L;
+         flowFilesIn = 0;
+         flowFilesOut = 0;
+         removedCount = 0;
+         removedBytes = 0L;
+         bytesRead.setValue(0L);
+         bytesWritten.setValue(0L);
+         connectionCounts.clear();
+         createdFlowFiles.clear();
+         removedFlowFiles.clear();
+         globalCounters.clear();
+         localCounters.clear();
+ 
+         generatedProvenanceEvents.clear();
+         forkEventBuilders.clear();
+         provenanceReporter.clear();
+ 
+         processingStartTime = System.nanoTime();
+     }
+ 
+     private void acknowledgeRecords() {
+         for (final Map.Entry<Connection, Set<FlowFileRecord>> entry : unacknowledgedFlowFiles.entrySet()) {
+             entry.getKey().getFlowFileQueue().acknowledge(entry.getValue());
+         }
+         unacknowledgedFlowFiles.clear();
+     }
+ 
+     private String summarizeEvents(final Checkpoint checkpoint) {
+         final Map<Relationship, Set<String>> transferMap = new HashMap<>(); // relationship to flowfile ID's
+         final Set<String> modifiedFlowFileIds = new HashSet<>();
+         int largestTransferSetSize = 0;
+ 
+         for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+             final FlowFile flowFile = entry.getKey();
+             final StandardRepositoryRecord record = entry.getValue();
+ 
+             final Relationship relationship = record.getTransferRelationship();
+             if (Relationship.SELF.equals(relationship)) {
+                 continue;
+             }
+ 
+             Set<String> transferIds = transferMap.get(relationship);
+             if (transferIds == null) {
+                 transferIds = new HashSet<>();
+                 transferMap.put(relationship, transferIds);
+             }
+             transferIds.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+             largestTransferSetSize = Math.max(largestTransferSetSize, transferIds.size());
+ 
+             final ContentClaim workingClaim = record.getWorkingClaim();
+             if (workingClaim != null && workingClaim != record.getOriginalClaim() && record.getTransferRelationship() != null) {
+                 modifiedFlowFileIds.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+             }
+         }
+ 
+         final int numRemoved = checkpoint.removedFlowFiles.size();
+         final int numModified = modifiedFlowFileIds.size();
+         final int numCreated = checkpoint.createdFlowFiles.size();
+ 
+         final StringBuilder sb = new StringBuilder(512);
+         if (!LOG.isDebugEnabled() && (largestTransferSetSize > VERBOSE_LOG_THRESHOLD || numModified > VERBOSE_LOG_THRESHOLD || numCreated > VERBOSE_LOG_THRESHOLD || numRemoved > VERBOSE_LOG_THRESHOLD)) {
+             if (numCreated > 0) {
+                 sb.append("created ").append(numCreated).append(" FlowFiles, ");
+             }
+             if (numModified > 0) {
+                 sb.append("modified ").append(modifiedFlowFileIds.size()).append(" FlowFiles, ");
+             }
+             if (numRemoved > 0) {
+                 sb.append("removed ").append(numRemoved).append(" FlowFiles, ");
+             }
+             for (final Map.Entry<Relationship, Set<String>> entry : transferMap.entrySet()) {
+                 if (entry.getKey() != null) {
+                     sb.append("Transferred ").append(entry.getValue().size()).append(" FlowFiles");
+ 
+                     final Relationship relationship = entry.getKey();
+                     if (relationship != Relationship.ANONYMOUS) {
+                         sb.append(" to '").append(relationship.getName()).append("', ");
+                     }
+                 }
+             }
+         } else {
+             if (numCreated > 0) {
+                 sb.append("created FlowFiles ").append(checkpoint.createdFlowFiles).append(", ");
+             }
+             if (numModified > 0) {
+                 sb.append("modified FlowFiles ").append(modifiedFlowFileIds).append(", ");
+             }
+             if (numRemoved > 0) {
+                 sb.append("removed FlowFiles ").append(checkpoint.removedFlowFiles).append(", ");
+             }
+             for (final Map.Entry<Relationship, Set<String>> entry : transferMap.entrySet()) {
+                 if (entry.getKey() != null) {
+                     sb.append("Transferred FlowFiles ").append(entry.getValue());
+ 
+                     final Relationship relationship = entry.getKey();
+                     if (relationship != Relationship.ANONYMOUS) {
+                         sb.append(" to '").append(relationship.getName()).append("', ");
+                     }
+                 }
+             }
+         }
+ 
+         if (sb.length() > 2 && sb.subSequence(sb.length() - 2, sb.length()).equals(", ")) {
+             sb.delete(sb.length() - 2, sb.length());
+         }
+ 
+         // don't add processing time if we did nothing, because we don't log the summary anyway
+         if (sb.length() > 0) {
+             final long processingNanos = checkpoint.processingTime;
+             sb.append(", Processing Time = ");
+             formatNanos(processingNanos, sb);
+         }
+ 
+         return sb.toString();
+     }
+ 
+     private void formatNanos(final long nanos, final StringBuilder sb) {
+         final long seconds = (nanos > 1000000000L) ? (nanos / 1000000000L) : 0L;
+         long millis = (nanos > 1000000L) ? (nanos / 1000000L) : 0L;;
+         final long nanosLeft = nanos % 1000000L;
+ 
+         if (seconds > 0) {
+             sb.append(seconds).append(" seconds");
+         }
+         if (millis > 0) {
+             if (seconds > 0) {
+                 sb.append(", ");
+                 millis -= seconds * 1000L;
+             }
+ 
+             sb.append(millis).append(" millis");
+         }
+         if (seconds == 0 && millis == 0) {
+             sb.append(nanosLeft).append(" nanos");
+         }
+ 
+         sb.append(" (").append(nanos).append(" nanos)");
+     }
+ 
+     private void incrementConnectionInputCounts(final Connection connection, final RepositoryRecord record) {
+         StandardFlowFileEvent connectionEvent = connectionCounts.get(connection);
+         if (connectionEvent == null) {
+             connectionEvent = new StandardFlowFileEvent(connection.getIdentifier());
+             connectionCounts.put(connection, connectionEvent);
+         }
+         connectionEvent.setContentSizeIn(connectionEvent.getContentSizeIn() + record.getCurrent().getSize());
+         connectionEvent.setFlowFilesIn(connectionEvent.getFlowFilesIn() + 1);
+     }
+ 
+     private void incrementConnectionOutputCounts(final Connection connection, final FlowFileRecord record) {
+         StandardFlowFileEvent connectionEvent = connectionCounts.get(connection);
+         if (connectionEvent == null) {
+             connectionEvent = new StandardFlowFileEvent(connection.getIdentifier());
+             connectionCounts.put(connection, connectionEvent);
+         }
+         connectionEvent.setContentSizeOut(connectionEvent.getContentSizeOut() + record.getSize());
+         connectionEvent.setFlowFilesOut(connectionEvent.getFlowFilesOut() + 1);
+     }
+ 
+     private void registerDequeuedRecord(final FlowFileRecord flowFile, final Connection connection) {
+         final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
+         records.put(flowFile, record);
+         flowFilesIn++;
+         contentSizeIn += flowFile.getSize();
+ 
+         Set<FlowFileRecord> set = unacknowledgedFlowFiles.get(connection);
+         if (set == null) {
+             set = new HashSet<>();
+             unacknowledgedFlowFiles.put(connection, set);
+         }
+         set.add(flowFile);
+ 
+         incrementConnectionOutputCounts(connection, flowFile);
+     }
+ 
+     @Override
+     public void adjustCounter(final String name, final long delta, final boolean immediate) {
+         if (immediate) {
+             context.adjustCounter(name, delta);
+             return;
+         }
+ 
+         adjustCounter(name, delta, localCounters);
+         adjustCounter(name, delta, globalCounters);
+     }
+ 
+     private void adjustCounter(final String name, final long delta, final Map<String, Long> map) {
+         Long curVal = map.get(name);
+         if (curVal == null) {
+             curVal = Long.valueOf(0L);
+         }
+ 
+         final long newValue = curVal.longValue() + delta;
+         map.put(name, Long.valueOf(newValue));
+     }
+ 
+     @Override
+     public FlowFile get() {
+         final List<Connection> connections = context.getPollableConnections();
+         final int numConnections = connections.size();
+         for (int numAttempts = 0; numAttempts < numConnections; numAttempts++) {
+             final Connection conn = connections.get(context.getNextIncomingConnectionIndex() % connections.size());
+             final Set<FlowFileRecord> expired = new HashSet<>();
+             final FlowFileRecord flowFile = conn.getFlowFileQueue().poll(expired);
+             removeExpired(expired, conn);
+ 
+             if (flowFile != null) {
+                 registerDequeuedRecord(flowFile, conn);
+                 return flowFile;
+             }
+         }
+ 
+         return null;
+     }
+ 
+     @Override
+     public List<FlowFile> get(final int maxResults) {
+         if (maxResults < 0) {
+             throw new IllegalArgumentException();
+         }
+         if (maxResults == 0) {
+             return Collections.emptyList();
+         }
+ 
+         return get(new QueuePoller() {
+             @Override
+             public List<FlowFileRecord> poll(final FlowFileQueue queue, final Set<FlowFileRecord> expiredRecords) {
+                 return queue.poll(new FlowFileFilter() {
+                     int polled = 0;
+ 
+                     @Override
+                     public FlowFileFilterResult filter(final FlowFile flowFile) {
+                         if (++polled <= maxResults) {
+                             return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
+                         } else {
+                             return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
+                         }
+                     }
+                 }, expiredRecords);
+             }
+         }, false);
+     }
+ 
+     @Override
+     public List<FlowFile> get(final FlowFileFilter filter) {
+         return get(new QueuePoller() {
+             @Override
+             public List<FlowFileRecord> poll(final FlowFileQueue queue, final Set<FlowFileRecord> expiredRecords) {
+                 return queue.poll(filter, expiredRecords);
+             }
+         }, true);
+     }
+ 
+     private List<FlowFile> get(final QueuePoller poller, final boolean lockAllQueues) {
+         final List<Connection> connections = context.getPollableConnections();
+         if (lockAllQueues) {
+             for (final Connection connection : connections) {
+                 connection.lock();
+             }
+         }
+ 
+         try {
+             for (final Connection conn : connections) {
+                 final Set<FlowFileRecord> expired = new HashSet<>();
+                 final List<FlowFileRecord> newlySelected = poller.poll(conn.getFlowFileQueue(), expired);
+                 removeExpired(expired, conn);
+ 
+                 if (newlySelected.isEmpty() && expired.isEmpty()) {
+                     continue;
+                 }
+ 
+                 for (final FlowFileRecord flowFile : newlySelected) {
+                     registerDequeuedRecord(flowFile, conn);
+                 }
+ 
+                 return new ArrayList<FlowFile>(newlySelected);
+             }
+ 
+             return new ArrayList<>();
+         } finally {
+             if (lockAllQueues) {
+                 for (final Connection connection : connections) {
+                     connection.unlock();
+                 }
+             }
+         }
+     }
+ 
+     @Override
+     public QueueSize getQueueSize() {
+         int flowFileCount = 0;
+         long byteCount = 0L;
+         for (final Connection conn : context.getPollableConnections()) {
+             final QueueSize queueSize = conn.getFlowFileQueue().getActiveQueueSize();
+             flowFileCount += queueSize.getObjectCount();
+             byteCount += queueSize.getByteCount();
+         }
+         return new QueueSize(flowFileCount, byteCount);
+     }
+ 
+     @Override
 -    public Set<Relationship> getAvailableRelationships() {
 -        return context.getAvailableRelationships();
 -    }
 -
 -    @Override
+     public FlowFile create() {
+         final Map<String, String> attrs = new HashMap<>();
+         attrs.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime()));
+         attrs.put(CoreAttributes.PATH.key(), DEFAULT_FLOWFILE_PATH);
+         attrs.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+ 
+         final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence())
+                 .addAttributes(attrs)
+                 .build();
+         final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
+         record.setWorking(fFile, attrs);
+         records.put(fFile, record);
+         createdFlowFiles.add(fFile.getAttribute(CoreAttributes.UUID.key()));
+         return fFile;
+     }
+ 
+     @Override
+     public FlowFile clone(final FlowFile example) {
+         return clone(example, 0L, example.getSize());
+     }
+ 
+     @Override
+     public FlowFile clone(final FlowFile example, final long offset, final long size) {
+         validateRecordState(example);
+         final StandardRepositoryRecord exampleRepoRecord = records.get(example);
+         final FlowFileRecord currRec = exampleRepoRecord.getCurrent();
+         final ContentClaim claim = exampleRepoRecord.getCurrentClaim();
+         if (offset + size > example.getSize()) {
+             throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + example.toString());
+         }
+ 
+         final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec);
+         builder.id(context.getNextFlowFileSequence());
+         builder.contentClaimOffset(currRec.getContentClaimOffset() + offset);
+         builder.size(size);
+ 
+         final String newUuid = UUID.randomUUID().toString();
+         builder.addAttribute(CoreAttributes.UUID.key(), newUuid);
+ 
+         final FlowFileRecord clone = builder.build();
+         if (claim != null) {
+             context.getContentRepository().incrementClaimaintCount(claim);
+         }
+         final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
+         record.setWorking(clone, CoreAttributes.UUID.key(), newUuid);
+         records.put(clone, record);
+ 
+         if (offset == 0L && size == example.getSize()) {
+             provenanceReporter.clone(example, clone);
+         } else {
+             registerForkEvent(example, clone);
+         }
+ 
+         return clone;
+     }
+ 
+     private void registerForkEvent(final FlowFile parent, final FlowFile child) {
+         ProvenanceEventBuilder eventBuilder = forkEventBuilders.get(parent);
+         if (eventBuilder == null) {
+             eventBuilder = context.getProvenanceRepository().eventBuilder();
+             eventBuilder.setEventType(ProvenanceEventType.FORK);
+ 
+             eventBuilder.setFlowFileEntryDate(parent.getEntryDate());
+             eventBuilder.setLineageIdentifiers(parent.getLineageIdentifiers());
+             eventBuilder.setLineageStartDate(parent.getLineageStartDate());
+             eventBuilder.setFlowFileUUID(parent.getAttribute(CoreAttributes.UUID.key()));
+ 
+             eventBuilder.setComponentId(context.getConnectable().getIdentifier());
+ 
+             final Connectable connectable = context.getConnectable();
+             final String processorType;
+             if (connectable instanceof ProcessorNode) {
+                 processorType = ((ProcessorNode) connectable).getProcessor().getClass().getSimpleName();
+             } else {
+                 processorType = connectable.getClass().getSimpleName();
+             }
+             eventBuilder.setComponentType(processorType);
+             eventBuilder.addParentFlowFile(parent);
+ 
+             updateEventContentClaims(eventBuilder, parent, records.get(parent));
+             forkEventBuilders.put(parent, eventBuilder);
+         }
+ 
+         eventBuilder.addChildFlowFile(child);
+     }
+ 
+     private void registerJoinEvent(final FlowFile child, final Collection<FlowFile> parents) {
+         final ProvenanceEventRecord eventRecord = provenanceReporter.generateJoinEvent(parents, child);
+         List<ProvenanceEventRecord> existingRecords = generatedProvenanceEvents.get(child);
+         if (existingRecords == null) {
+             existingRecords = new ArrayList<>();
+             generatedProvenanceEvents.put(child, existingRecords);
+         }
+         existingRecords.add(eventRecord);
+     }
+ 
+     @Override
+     public FlowFile penalize(final FlowFile flowFile) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+         final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS);
+         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).penaltyExpirationTime(expirationEpochMillis).build();
+         record.setWorking(newFile);
+         return newFile;
+     }
+ 
+     @Override
+     public FlowFile putAttribute(final FlowFile flowFile, final String key, final String value) {
+         validateRecordState(flowFile);
+ 
+         if (CoreAttributes.UUID.key().equals(key)) {
+             return flowFile;
+         }
+ 
+         final StandardRepositoryRecord record = records.get(flowFile);
+         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttribute(key, value).build();
+         record.setWorking(newFile, key, value);
+ 
+         return newFile;
+     }
+ 
+     @Override
+     public FlowFile putAllAttributes(final FlowFile flowFile, final Map<String, String> attributes) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+ 
+         final String originalUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+ 
+         final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttributes(attributes);
+         // Ignore the uuid attribute, if passed in
+         ffBuilder.addAttribute(CoreAttributes.UUID.key(), originalUuid);
+         final FlowFileRecord newFile = ffBuilder.build();
+ 
+         record.setWorking(newFile, attributes);
+         return newFile;
+     }
+ 
+     @Override
+     public FlowFile removeAttribute(final FlowFile flowFile, final String key) {
+         validateRecordState(flowFile);
+ 
+         if (CoreAttributes.UUID.key().equals(key)) {
+             return flowFile;
+         }
+ 
+         final StandardRepositoryRecord record = records.get(flowFile);
+         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(key).build();
+         record.setWorking(newFile, key, null);
+         return newFile;
+     }
+ 
+     @Override
+     public FlowFile removeAllAttributes(final FlowFile flowFile, final Set<String> keys) {
+         validateRecordState(flowFile);
+ 
+         if (keys == null) {
+             return flowFile;
+         }
+ 
+         final Set<String> keysToRemove;
+         if (keys.contains(CoreAttributes.UUID.key())) {
+             keysToRemove = new HashSet<>(keys);
+             keysToRemove.remove(CoreAttributes.UUID.key());
+         } else {
+             keysToRemove = keys;
+         }
+ 
+         final StandardRepositoryRecord record = records.get(flowFile);
+         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keys).build();
+ 
+         final Map<String, String> updatedAttrs = new HashMap<>();
+         for (final String key : keys) {
+             updatedAttrs.put(key, null);
+         }
+ 
+         record.setWorking(newFile, updatedAttrs);
+         return newFile;
+     }
+ 
+     @Override
+     public FlowFile removeAllAttributes(final FlowFile flowFile, final Pattern keyPattern) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+         final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keyPattern).build();
+ 
+         if (keyPattern == null) {
+             record.setWorking(newFile);
+         } else {
+             final Map<String, String> curAttrs = record.getCurrent().getAttributes();
+ 
+             final Map<String, String> removed = new HashMap<>();
+             for (final String key : curAttrs.keySet()) {
+                 if (CoreAttributes.UUID.key().equals(key)) {
+                     continue;
+                 }
+ 
+                 if (keyPattern.matcher(key).matches()) {
+                     removed.put(key, null);
+                 }
+             }
+ 
+             record.setWorking(newFile, removed);
+         }
+ 
+         return newFile;
+     }
+ 
+     @Override
+     public void transfer(final FlowFile flowFile, final Relationship relationship) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+         record.setTransferRelationship(relationship);
+         final int numDestinations = context.getConnections(relationship).size();
+         final int multiplier = Math.max(1, numDestinations);
+ 
+         boolean autoTerminated = false;
+         boolean selfRelationship = false;
+         if (numDestinations == 0 && context.getConnectable().isAutoTerminated(relationship)) {
+             // auto terminated.
+             autoTerminated = true;
+         } else if (numDestinations == 0 && relationship == Relationship.SELF) {
+             selfRelationship = true;
+         }
+ 
+         if (autoTerminated) {
+             removedCount += multiplier;
+             removedBytes += flowFile.getSize();
+         } else if (!selfRelationship) {
+             flowFilesOut += multiplier;
+             contentSizeOut += flowFile.getSize() * multiplier;
+         }
+     }
+ 
+     @Override
+     public void transfer(final FlowFile flowFile) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+         if (record.getOriginalQueue() == null) {
+             throw new IllegalArgumentException("Cannot transfer FlowFiles that are created in this Session back to self");
+         }
+         record.setTransferRelationship(Relationship.SELF);
+     }
+ 
+     @Override
+     public void transfer(final Collection<FlowFile> flowFiles) {
+         for (final FlowFile flowFile : flowFiles) {
+             transfer(flowFile);
+         }
+     }
+ 
+     @Override
+     public void transfer(final Collection<FlowFile> flowFiles, final Relationship relationship) {
+         validateRecordState(flowFiles);
+ 
+         boolean autoTerminated = false;
+         boolean selfRelationship = false;
+         final int numDestinations = context.getConnections(relationship).size();
+         if (numDestinations == 0 && context.getConnectable().isAutoTerminated(relationship)) {
+             // auto terminated.
+             autoTerminated = true;
+         } else if (numDestinations == 0 && relationship == Relationship.SELF) {
+             selfRelationship = true;
+         }
+ 
+         final int multiplier = Math.max(1, numDestinations);
+ 
+         long contentSize = 0L;
+         for (final FlowFile flowFile : flowFiles) {
+             final StandardRepositoryRecord record = records.get(flowFile);
+             record.setTransferRelationship(relationship);
+             contentSize += flowFile.getSize() * multiplier;
+         }
+ 
+         if (autoTerminated) {
+             removedCount += multiplier * flowFiles.size();
+             removedBytes += contentSize;
+         } else if (!selfRelationship) {
+             flowFilesOut += multiplier * flowFiles.size();
+             contentSizeOut += multiplier * contentSize;
+         }
+     }
+ 
+     @Override
+     public void remove(final FlowFile flowFile) {
+         validateRecordState(flowFile);
+         final StandardRepositoryRecord record = records.get(flowFile);
+         record.markForDelete();
+         removedFlowFiles.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+ 
+         // if original connection is null, the FlowFile was created in this session, so we
+         // do not want to count it toward the removed count.
+         if (record.getOriginalQueue() == null) {
+             // if we've generated any Fork events, remove them because the FlowFile was created
+             // and then removed in this session.
+             generatedProvenanceEvents.remove(flowFile);
+             removeForkEvents(flowFile);
+         } else {
+             removedCount++;
+             removedBytes += flowFile.getSize();
+             provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key()));
+         }
+     }
+ 
+     @Override
+     public void remove(final Collection<FlowFile> flowFiles) {
+         validateRecordState(flowFiles);
+         for (final FlowFile flowFile : flowFiles) {
+             final StandardRepositoryRecord record = records.get(flowFile);
+             record.markForDelete();
+             removedFlowFiles.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+ 
+             // if original connection is null, the FlowFile was created in this session, so we
+             // do not want to count it toward the removed count.
+             if (record.getOriginalQueue() == null) {
+                 generatedProvenanceEvents.remove(flowFile);
+                 removeForkEvents(flowFile);
+             } else {
+                 removedCount++;
+                 removedBytes += flowFile.getSize();
+                 provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key()));
+             }
+         }
+     }
+ 
+     private void removeForkEvents(final FlowFile flowFile) {
+         for (final ProvenanceEventBuilder builder : forkEventBuilders.values()) {
+             final ProvenanceEventRecord event = builder.build();
+ 
+             if (event.getEventType() == ProvenanceEventType.FORK) {
+                 builder.removeChildFlowFile(flowFile);
+             }
+         }
+     }
+ 
+     public void expireFlowFiles() {
+         final Set<FlowFileRecord> expired = new HashSet<>();
+         final FlowFileFilter filter = new FlowFileFilter() {
+             @Override
+             public FlowFileFilterResult filter(final FlowFile flowFile) {
+                 return FlowFileFilterResult.REJECT_AND_CONTINUE;
+             }
+         };
+ 
+         for (final Connection conn : context.getConnectable().getIncomingConnections()) {
+             do {
+                 expired.clear();
+                 conn.getFlowFileQueue().poll(filter, expired);
+                 removeExpired(expired, conn);
+             } while (!expired.isEmpty());
+         }
+     }
+ 
+     private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) {
+         if (flowFiles.isEmpty()) {
+             return;
+         }
+ 
+         LOG.info("{} {} FlowFiles have expired and will be removed", new Object[]{this, flowFiles.size()});
+         final List<RepositoryRecord> expiredRecords = new ArrayList<>(flowFiles.size());
+ 
+         final String processorType;
+         final Connectable connectable = context.getConnectable();
+         if (connectable instanceof ProcessorNode) {
+             final ProcessorNode procNode = (ProcessorNode) connectable;
+             processorType = procNode.getProcessor().getClass().getSimpleName();
+         } else {
+             processorType = connectable.getClass().getSimpleName();
+         }
+ 
+         final StandardProvenanceReporter expiredReporter = new StandardProvenanceReporter(connectable.getIdentifier(),
+                 processorType, context.getProvenanceRepository(), this);
+ 
+         final Map<String, FlowFileRecord> recordIdMap = new HashMap<>();
+         for (final FlowFileRecord flowFile : flowFiles) {
+             recordIdMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+ 
+             final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
+             record.markForDelete();
+             expiredRecords.add(record);
+             expiredReporter.expire(flowFile, "Expiration Threshold = " + connection.getFlowFileQueue().getFlowFileExpiration());
+             removeContent(flowFile.getContentClaim());
+ 
+             final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
+             final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
+             LOG.info("{} terminated by {} due to FlowFile expiration; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
+         }
+ 
+         try {
+             final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
+                 @Override
+                 public Iterator<ProvenanceEventRecord> iterator() {
+                     final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
+                     final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() {
+                         @Override
+                         public boolean hasNext() {
+                             return expiredEventIterator.hasNext();
+                         }
+ 
+                         @Override
+                         public ProvenanceEventRecord next() {
+                             final ProvenanceEventRecord event = expiredEventIterator.next();
+                             final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
+                             final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
+                             if (record == null) {
+                                 return null;
+                             }
+ 
+                             final ContentClaim claim = record.getContentClaim();
+                             if (claim != null) {
+                                 enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+                                 enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+                             }
+ 
+                             enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
+                             return enriched.build();
+                         }
+ 
+                         @Override
+                         public void remove() {
+                             throw new UnsupportedOperationException();
+                         }
+                     };
+ 
+                     return enrichingIterator;
+                 }
+             };
+ 
+             context.getProvenanceRepository().registerEvents(iterable);
+             context.getFlowFileRepository().updateRepository(expiredRecords);
+         } catch (final IOException e) {
+             LOG.error("Failed to update FlowFile Repository to record expired records due to {}", e);
+         }
+ 
+     }
+ 
+     private InputStream getInputStream(final FlowFile flowFile, final ContentClaim claim, final long offset) throws ContentNotFoundException {
+         // If there's no content, don't bother going to the Content Repository because it is generally expensive and we know
+         // that there is no actual content.
+         if (flowFile.getSize() == 0L) {
+             return new ByteArrayInputStream(new byte[0]);
+         }
+ 
+         try {
+             // If the recursion set is empty, we can use the same input stream that we already have open. However, if
+             // the recursion set is NOT empty, we can't do this because we may be reading the input of FlowFile 1 while in the
+             // callback for reading FlowFile 1 and if we used the same stream we'd be destroying the ability to read from FlowFile 1.
+             if (recursionSet.isEmpty()) {
+                 if (currentReadClaim == claim) {
+                     if (currentReadClaimStream != null && currentReadClaimStream.getStreamLocation() <= offset) {
+                         final long bytesToSkip = offset - currentReadClaimStream.getStreamLocation();
+                         if (bytesToSkip > 0) {
+                             StreamUtils.skip(currentReadClaimStream, bytesToSkip);
+                         }
+ 
+                         return new NonCloseableInputStream(currentReadClaimStream);
+                     }
+                 }
+ 
+                 final InputStream rawInStream = context.getContentRepository().read(claim);
+ 
+                 if (currentReadClaimStream != null) {
+                     currentReadClaimStream.close();
+                 }
+ 
+                 currentReadClaim = claim;
+                 currentReadClaimStream = new ByteCountingInputStream(rawInStream, new LongHolder(0L));
+                 StreamUtils.skip(currentReadClaimStream, offset);
+ 
+                 // Use a non-closeable stream because we want to keep it open after the callback has finished so that we can
+                 // reuse the same InputStream for the next FlowFile
+                 return new NonCloseableInputStream(currentReadClaimStream);
+             } else {
+                 final InputStream rawInStream = context.getContentRepository().read(claim);
+                 StreamUtils.skip(rawInStream, offset);
+                 return rawInStream;
+             }
+         } catch (final ContentNotFoundException cnfe) {
+             throw cnfe;
+         } catch (final EOFException eof) {
+             throw new ContentNotFoundException(claim, eof);
+         } catch (final IOException ioe) {
+             throw new FlowFileAccessException("Failed to read content of " + flowFile, ioe);
+         }
+     }
+ 
+     @Override
+     public void read(final FlowFile source, final InputStreamCallback reader) {
+         validateRecordState(source);
+         final StandardRepositoryRecord record = records.get(source);
+ 
+         try {
+             ensureNotAppending(record.getCurrentClaim());
+         } catch (final IOException e) {
+             throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e);
+         }
+ 
+         try (final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset());
+                 final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
+                 final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn, this.bytesRead)) {
+ 
+             // We want to differentiate between IOExceptions thrown by the repository and IOExceptions thrown from
+             // Processor code. As a result, as have the FlowFileAccessInputStream that catches IOException from the repository
+             // and translates into either FlowFileAccessException or ContentNotFoundException. We keep track of any
+             // ContentNotFoundException because if it is thrown, the Processor code may catch it and do something else with it
+             // but in reality, if it is thrown, we want to know about it and handle it, even if the Processor code catches it.
+             final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim());
+             boolean cnfeThrown = false;
+ 
+             try {
+                 recursionSet.add(source);
+                 reader.process(ffais);
+             } catch (final ContentNotFoundException cnfe) {
+                 cnfeThrown = true;
+                 throw cnfe;
+             } finally {
+                 recursionSet.remove(source);
+ 
+                 // if cnfeThrown is true, we don't need to re-thrown the Exception; it will propagate.
+                 if (!cnfeThrown && ffais.getContentNotFoundException() != null) {
+                     throw ffais.getContentNotFoundException();
+                 }
+             }
+         } catch (final ContentNotFoundException nfe) {
+             handleContentNotFound(nfe, record);
+         } catch (final IOException ex) {
+             throw new ProcessException("IOException thrown from " + connectableDescription + ": " + ex.toString(), ex);
+         }
+     }
+ 
+     @Override
+     public FlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {
+         return merge(sources, destination, null, null, null);
+     }
+ 
+     @Override
+     public FlowFile merge(final Collection<FlowFile> sources, final FlowFile destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+         validateRecordState(sources);
+         validateRecordState(destination);
+         if (sources.contains(destination)) {
+             throw new IllegalArgumentException("Destination cannot be within sources");
+         }
+ 
+         final Collection<StandardRepositoryRecord> sourceRecords = new ArrayList<>();
+         for (final FlowFile source : sources) {
+             final StandardRepositoryRecord record = records.get(source);
+             sourceRecords.add(record);
+ 
+             try {
+                 ensureNotAppending(record.getCurrentClaim());
+             } catch (final IOException e) {
+                 throw new FlowFileAccessException("Unable to read from source " + source + " due to " + e.toString(), e);
+             }
+         }
+ 
+         final StandardRepositoryRecord destinationRecord = records.get(destination);
+         final ContentRepository contentRepo = context.getContentRepository();
+         final ContentClaim newClaim;
+         try {
+             newClaim = contentRepo.create(context.getConnectable().isLossTolerant());
+             claimLog.debug("Creating ContentClaim {} for 'merge' for {}", newClaim, desti

<TRUNCATED>

[2/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index 0000000,6e5f65d..15591d7
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@@ -1,0 -1,261 +1,284 @@@
+ /*
+  * 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.util;
+ 
+ import static java.util.Objects.requireNonNull;
+ 
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Objects;
++import java.util.Set;
+ 
+ import org.apache.nifi.components.ConfigurableComponent;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
++import org.apache.nifi.processor.Processor;
++import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.SchedulingContext;
+ import org.junit.Assert;
+ 
+ public class MockProcessContext extends MockControllerServiceLookup implements SchedulingContext, ControllerServiceLookup {
+ 
+     private final ConfigurableComponent component;
+     private final Map<PropertyDescriptor, String> properties = new HashMap<>();
+ 
+     private String annotationData = null;
+     private boolean yieldCalled = false;
+     private boolean enableExpressionValidation = false;
+     private boolean allowExpressionValidation = true;
+ 
++    private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
++
+     /**
+      * Creates a new MockProcessContext for the given Processor
+      *
+      * @param component
+      */
+     public MockProcessContext(final ConfigurableComponent component) {
+         this.component = Objects.requireNonNull(component);
+     }
+ 
+     public MockProcessContext(final ControllerService component, final MockProcessContext context) {
+         this(component);
+ 
+         try {
+             annotationData = context.getControllerServiceAnnotationData(component);
+             final Map<PropertyDescriptor, String> props = context.getControllerServiceProperties(component);
+             properties.putAll(props);
+         } catch (IllegalArgumentException e) {
+             // do nothing...the service is being loaded
+         }
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+         return getProperty(descriptor.getName());
+     }
+ 
+     @Override
+     public PropertyValue getProperty(final String propertyName) {
+         final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
+         if (descriptor == null) {
+             return null;
+         }
+ 
+         final String setPropertyValue = properties.get(descriptor);
+         final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
+         return new MockPropertyValue(propValue, this, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
+     }
+ 
+     @Override
+     public PropertyValue newPropertyValue(final String rawValue) {
+         return new MockPropertyValue(rawValue, this);
+     }
+ 
+     public ValidationResult setProperty(final String propertyName, final String propertyValue) {
+         return setProperty(new PropertyDescriptor.Builder().name(propertyName).build(), propertyValue);
+     }
+ 
+     /**
+      * Updates the value of the property with the given PropertyDescriptor to
+      * the specified value IF and ONLY IF the value is valid according to the
+      * descriptor's validator. Otherwise, the property value is not updated. In
+      * either case, the ValidationResult is returned, indicating whether or not
+      * the property is valid
+      *
+      * @param descriptor
+      * @param value
+      * @return
+      */
+     public ValidationResult setProperty(final PropertyDescriptor descriptor, final String value) {
+         requireNonNull(descriptor);
+         requireNonNull(value, "Cannot set property to null value; if the intent is to remove the property, call removeProperty instead");
+         final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
+ 
+         final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this));
+         String oldValue = properties.put(fullyPopulatedDescriptor, value);
+         if (oldValue == null) {
+             oldValue = fullyPopulatedDescriptor.getDefaultValue();
+         }
+         if ((value == null && oldValue != null) || (value != null && !value.equals(oldValue))) {
+             component.onPropertyModified(fullyPopulatedDescriptor, oldValue, value);
+         }
+ 
+         return result;
+     }
+ 
+     public boolean removeProperty(final PropertyDescriptor descriptor) {
+         Objects.requireNonNull(descriptor);
+         final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
+         String value = null;
+         if (!fullyPopulatedDescriptor.isRequired() && (value = properties.remove(fullyPopulatedDescriptor)) != null) {
+             component.onPropertyModified(fullyPopulatedDescriptor, value, null);
+             return true;
+         }
+         return false;
+     }
+ 
+     @Override
+     public void yield() {
+         yieldCalled = true;
+     }
+ 
+     public boolean isYieldCalled() {
+         return yieldCalled;
+     }
+ 
+     public void addControllerService(final String serviceIdentifier, final ControllerService controllerService, final Map<PropertyDescriptor, String> properties, final String annotationData) {
+         requireNonNull(controllerService);
+         final ControllerServiceConfiguration config = addControllerService(controllerService);
+         config.setProperties(properties);
+         config.setAnnotationData(annotationData);
+     }
+ 
+     @Override
+     public int getMaxConcurrentTasks() {
+         return 1;
+     }
+ 
+     public void setAnnotationData(final String annotationData) {
+         this.annotationData = annotationData;
+     }
+ 
+     @Override
+     public String getAnnotationData() {
+         return annotationData;
+     }
+ 
+     @Override
+     public Map<PropertyDescriptor, String> getProperties() {
+         final List<PropertyDescriptor> supported = component.getPropertyDescriptors();
+         if (supported == null || supported.isEmpty()) {
+             return Collections.unmodifiableMap(properties);
+         } else {
+             final Map<PropertyDescriptor, String> props = new LinkedHashMap<>();
+             for (final PropertyDescriptor descriptor : supported) {
+                 props.put(descriptor, null);
+             }
+             props.putAll(properties);
+             return props;
+         }
+     }
+ 
+     /**
+      * Validates the current properties, returning ValidationResults for any
+      * invalid properties. All processor defined properties will be validated.
+      * If they are not included in the in the purposed configuration, the
+      * default value will be used.
+      *
+      * @return Collection of validation result objects for any invalid findings
+      * only. If the collection is empty then the processor is valid. Guaranteed
+      * non-null
+      */
+     public Collection<ValidationResult> validate() {
+         return component.validate(new MockValidationContext(this));
+     }
+ 
+     public boolean isValid() {
+         for (final ValidationResult result : validate()) {
+             if (!result.isValid()) {
+                 return false;
+             }
+         }
+ 
+         return true;
+     }
+ 
+     public void assertValid() {
+         final StringBuilder sb = new StringBuilder();
+         int failureCount = 0;
+ 
+         for (final ValidationResult result : validate()) {
+             if (!result.isValid()) {
+                 sb.append(result.toString()).append("\n");
+                 failureCount++;
+             }
+         }
+ 
+         if (failureCount > 0) {
+             Assert.fail("Processor has " + failureCount + " validation failures:\n" + sb.toString());
+         }
+     }
+ 
+     @Override
+     public String encrypt(final String unencrypted) {
+         return "enc{" + unencrypted + "}";
+     }
+ 
+     @Override
+     public String decrypt(final String encrypted) {
+         if (encrypted.startsWith("enc{") && encrypted.endsWith("}")) {
+             return encrypted.substring(4, encrypted.length() - 2);
+         }
+         return encrypted;
+     }
+ 
+     public void setValidateExpressionUsage(final boolean validate) {
+         allowExpressionValidation = validate;
+     }
+ 
+     public void enableExpressionValidation() {
+         enableExpressionValidation = true;
+     }
+ 
+     public void disableExpressionValidation() {
+         enableExpressionValidation = false;
+     }
+ 
+     Map<PropertyDescriptor, String> getControllerServiceProperties(final ControllerService controllerService) {
+         return super.getConfiguration(controllerService.getIdentifier()).getProperties();
+     }
+ 
+     String getControllerServiceAnnotationData(final ControllerService controllerService) {
+         return super.getConfiguration(controllerService.getIdentifier()).getAnnotationData();
+     }
+ 
+     @Override
+     public ControllerServiceLookup getControllerServiceLookup() {
+         return this;
+     }
+ 
+     @Override
+     public void leaseControllerService(final String identifier) {
+     }
+ 
++    public Set<Relationship> getAvailableRelationships() {
++        if ( !(component instanceof Processor) ) {
++            return Collections.emptySet();
++        }
++        
++        final Set<Relationship> relationships = new HashSet<>(((Processor) component).getRelationships());
++        relationships.removeAll(unavailableRelationships);
++        return relationships;
++    }
++
++    public void setUnavailableRelationships(final Set<Relationship> relationships) {
++        this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
++    }
++
++    public Set<Relationship> getUnavailableRelationships() {
++        return unavailableRelationships;
++    }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index 0000000,552780c..ea55b34
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@@ -1,0 -1,1010 +1,1006 @@@
+ /*
+  * 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.util;
+ 
+ import java.io.ByteArrayInputStream;
+ import java.io.ByteArrayOutputStream;
+ import java.io.File;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Files;
+ import java.nio.file.OpenOption;
+ import java.nio.file.Path;
+ import java.nio.file.StandardOpenOption;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Objects;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.regex.Pattern;
+ 
+ import org.junit.Assert;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ 
+ public class MockProcessSession implements ProcessSession {
+ 
+     private final Map<Relationship, List<MockFlowFile>> transferMap = new ConcurrentHashMap<>();
+     private final MockFlowFileQueue processorQueue;
+     private final Set<Long> beingProcessed = new HashSet<>();
+ 
+     private final Map<Long, MockFlowFile> currentVersions = new HashMap<>();
+     private final Map<Long, MockFlowFile> originalVersions = new HashMap<>();
+     private final SharedSessionState sharedState;
+     private final Map<String, Long> counterMap = new HashMap<>();
+ 
+     private boolean committed = false;
+     private boolean rolledback = false;
+     private int removedCount = 0;
+ 
+     public MockProcessSession(final SharedSessionState sharedState) {
+         this.sharedState = sharedState;
+         this.processorQueue = sharedState.getFlowFileQueue();
+     }
+ 
+     @Override
+     public void adjustCounter(final String name, final long delta, final boolean immediate) {
+         if (immediate) {
+             sharedState.adjustCounter(name, delta);
+             return;
+         }
+ 
+         Long counter = counterMap.get(name);
+         if (counter == null) {
+             counter = delta;
+             counterMap.put(name, counter);
+             return;
+         }
+ 
+         counter = counter + delta;
+         counterMap.put(name, counter);
+     }
+ 
+     @Override
+     public MockFlowFile clone(final FlowFile flowFile) {
+         validateState(flowFile);
+         final MockFlowFile newFlowFile = new MockFlowFile(sharedState.nextFlowFileId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         beingProcessed.add(newFlowFile.getId());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile clone(final FlowFile flowFile, final long offset, final long size) {
+         validateState(flowFile);
+         if (offset + size > flowFile.getSize()) {
+             throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + flowFile.toString());
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(sharedState.nextFlowFileId(), flowFile);
+         final byte[] newContent = Arrays.copyOfRange(((MockFlowFile) flowFile).getData(), (int) offset, (int) (offset + size));
+         newFlowFile.setData(newContent);
+ 
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         beingProcessed.add(newFlowFile.getId());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public void commit() {
+         if (!beingProcessed.isEmpty()) {
+             throw new FlowFileHandlingException("Cannot commit session because the following FlowFiles have not been removed or transferred: " + beingProcessed);
+         }
+         committed = true;
+         beingProcessed.clear();
+         currentVersions.clear();
+         originalVersions.clear();
+ 
+         for (final Map.Entry<String, Long> entry : counterMap.entrySet()) {
+             sharedState.adjustCounter(entry.getKey(), entry.getValue());
+         }
+ 
+         counterMap.clear();
+     }
+ 
+     /**
+      * Clear the 'committed' flag so that we can test that the next iteration of
+      * {@link nifi.processor.Processor#onTrigger} commits or rolls back the
+      * session
+      */
+     public void clearCommited() {
+         committed = false;
+     }
+ 
+     /**
+      * Clear the 'rolledBack' flag so that we can test that the next iteration
+      * of {@link nifi.processor.Processor#onTrigger} commits or rolls back the
+      * session
+      */
+     public void clearRollback() {
+         rolledback = false;
+     }
+ 
+     @Override
+     public MockFlowFile create() {
+         final MockFlowFile flowFile = new MockFlowFile(sharedState.nextFlowFileId());
+         currentVersions.put(flowFile.getId(), flowFile);
+         beingProcessed.add(flowFile.getId());
+         return flowFile;
+     }
+ 
+     @Override
+     public MockFlowFile create(final FlowFile flowFile) {
+         MockFlowFile newFlowFile = create();
+         newFlowFile = (MockFlowFile) inheritAttributes(flowFile, newFlowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         beingProcessed.add(newFlowFile.getId());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile create(final Collection<FlowFile> flowFiles) {
+         MockFlowFile newFlowFile = create();
+         newFlowFile = (MockFlowFile) inheritAttributes(flowFiles, newFlowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         beingProcessed.add(newFlowFile.getId());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public void exportTo(final FlowFile flowFile, final OutputStream out) {
+         validateState(flowFile);
+         if (flowFile == null || out == null) {
+             throw new IllegalArgumentException("arguments cannot be null");
+         }
+ 
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+ 
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         try {
+             out.write(mock.getData());
+         } catch (IOException e) {
+             throw new FlowFileAccessException(e.toString(), e);
+         }
+     }
+ 
+     @Override
+     public void exportTo(final FlowFile flowFile, final Path path, final boolean append) {
+         validateState(flowFile);
+         if (flowFile == null || path == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+ 
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final OpenOption mode = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE;
+ 
+         try (final OutputStream out = Files.newOutputStream(path, mode)) {
+             out.write(mock.getData());
+         } catch (final IOException e) {
+             throw new FlowFileAccessException(e.toString(), e);
+         }
+     }
+ 
+     @Override
+     public MockFlowFile get() {
+         final MockFlowFile flowFile = processorQueue.poll();
+         if (flowFile != null) {
+             beingProcessed.add(flowFile.getId());
+             currentVersions.put(flowFile.getId(), flowFile);
+             originalVersions.put(flowFile.getId(), flowFile);
+         }
+         return flowFile;
+     }
+ 
+     @Override
+     public List<FlowFile> get(final int maxResults) {
+         final List<FlowFile> flowFiles = new ArrayList<>(Math.min(500, maxResults));
+         for (int i = 0; i < maxResults; i++) {
+             final MockFlowFile nextFlowFile = get();
+             if (nextFlowFile == null) {
+                 return flowFiles;
+             }
+ 
+             flowFiles.add(nextFlowFile);
+         }
+ 
+         return flowFiles;
+     }
+ 
+     @Override
+     public List<FlowFile> get(final FlowFileFilter filter) {
+         final List<FlowFile> flowFiles = new ArrayList<>();
+         final List<MockFlowFile> unselected = new ArrayList<>();
+ 
+         while (true) {
+             final MockFlowFile flowFile = processorQueue.poll();
+             if (flowFile == null) {
+                 break;
+             }
+ 
+             final FlowFileFilter.FlowFileFilterResult filterResult = filter.filter(flowFile);
+             if (filterResult.isAccept()) {
+                 flowFiles.add(flowFile);
+ 
+                 beingProcessed.add(flowFile.getId());
+                 currentVersions.put(flowFile.getId(), flowFile);
+                 originalVersions.put(flowFile.getId(), flowFile);
+             } else {
+                 unselected.add(flowFile);
+             }
+ 
+             if (!filterResult.isContinue()) {
+                 break;
+             }
+         }
+ 
+         processorQueue.addAll(unselected);
+         return flowFiles;
+     }
+ 
+     @Override
+     public QueueSize getQueueSize() {
+         return processorQueue.size();
+     }
+ 
+     @Override
+     public MockFlowFile importFrom(final InputStream in, final FlowFile flowFile) {
+         validateState(flowFile);
+         if (in == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         try {
+             final byte[] data = readFully(in);
+             newFlowFile.setData(data);
+             return newFlowFile;
+         } catch (final IOException e) {
+             throw new FlowFileAccessException(e.toString(), e);
+         }
+     }
+ 
+     @Override
+     public MockFlowFile importFrom(final Path path, final boolean keepSourceFile, final FlowFile flowFile) {
+         validateState(flowFile);
+         if (path == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+         MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         try {
+             Files.copy(path, baos);
+         } catch (final IOException e) {
+             throw new FlowFileAccessException(e.toString(), e);
+         }
+ 
+         newFlowFile.setData(baos.toByteArray());
+         newFlowFile = putAttribute(newFlowFile, CoreAttributes.FILENAME.key(), path.getFileName().toString());
+         return newFlowFile;
+     }
+ 
 -    @Override
 -    public Set<Relationship> getAvailableRelationships() {
 -        return sharedState.getAvailableRelationships();
 -    }
+ 
+     @Override
+     public MockFlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {
+         for (final FlowFile source : sources) {
+             validateState(source);
+         }
+         validateState(destination);
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         for (final FlowFile flowFile : sources) {
+             final MockFlowFile mock = (MockFlowFile) flowFile;
+             final byte[] data = mock.getData();
+             try {
+                 baos.write(data);
+             } catch (final IOException e) {
+                 throw new AssertionError("Failed to write to BAOS");
+             }
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(destination.getId(), destination);
+         newFlowFile.setData(baos.toByteArray());
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile putAllAttributes(final FlowFile flowFile, final Map<String, String> attrs) {
+         validateState(flowFile);
+         if (attrs == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot update attributes of a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         newFlowFile.putAttributes(attrs);
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile putAttribute(final FlowFile flowFile, final String attrName, final String attrValue) {
+         validateState(flowFile);
+         if (attrName == null || attrValue == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot update attributes of a flow file that I did not create");
+         }
+ 
+         if ("uuid".equals(attrName)) {
+             Assert.fail("Should not be attempting to set FlowFile UUID via putAttribute. This will be ignored in production");
+         }
+ 
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         final Map<String, String> attrs = new HashMap<>();
+         attrs.put(attrName, attrValue);
+         newFlowFile.putAttributes(attrs);
+         return newFlowFile;
+     }
+ 
+     @Override
+     public void read(final FlowFile flowFile, final InputStreamCallback callback) {
+         if (callback == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+ 
+         validateState(flowFile);
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final ByteArrayInputStream bais = new ByteArrayInputStream(mock.getData());
+         try {
+             callback.process(bais);
+         } catch (IOException e) {
+             throw new ProcessException(e.toString(), e);
+         }
+     }
+ 
+     @Override
+     public void remove(final FlowFile flowFile) {
+         validateState(flowFile);
+         final Iterator<Long> itr = beingProcessed.iterator();
+         while (itr.hasNext()) {
+             final Long ffId = itr.next();
+             if (ffId != null && ffId.equals(flowFile.getId())) {
+                 itr.remove();
+                 beingProcessed.remove(ffId);
+                 removedCount++;
+                 currentVersions.remove(ffId);
+                 return;
+             }
+         }
+ 
+         throw new ProcessException(flowFile + " not found in queue");
+     }
+ 
+     @Override
+     public void remove(final Collection<FlowFile> flowFiles) {
+         for (final FlowFile flowFile : flowFiles) {
+             validateState(flowFile);
+         }
+ 
+         for (final FlowFile flowFile : flowFiles) {
+             remove(flowFile);
+         }
+     }
+ 
+     @Override
+     public MockFlowFile removeAllAttributes(final FlowFile flowFile, final Set<String> attrNames) {
+         validateState(flowFile);
+         if (attrNames == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         newFlowFile.removeAttributes(attrNames);
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile removeAllAttributes(final FlowFile flowFile, final Pattern keyPattern) {
+         validateState(flowFile);
+         if (flowFile == null) {
+             throw new IllegalArgumentException("flowFile cannot be null");
+         }
+         if (keyPattern == null) {
+             return (MockFlowFile) flowFile;
+         }
+ 
+         final Set<String> attrsToRemove = new HashSet<>();
+         for (final String key : flowFile.getAttributes().keySet()) {
+             if (keyPattern.matcher(key).matches()) {
+                 attrsToRemove.add(key);
+             }
+         }
+ 
+         return removeAllAttributes(flowFile, attrsToRemove);
+     }
+ 
+     @Override
+     public MockFlowFile removeAttribute(final FlowFile flowFile, final String attrName) {
+         validateState(flowFile);
+         if (attrName == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         final Set<String> attrNames = new HashSet<>();
+         attrNames.add(attrName);
+         newFlowFile.removeAttributes(attrNames);
+         return newFlowFile;
+     }
+ 
+     @Override
+     public void rollback() {
+         rollback(false);
+     }
+ 
+     @Override
+     public void rollback(final boolean penalize) {
+         for (final List<MockFlowFile> list : transferMap.values()) {
+             for (final MockFlowFile flowFile : list) {
+                 processorQueue.offer(flowFile);
+             }
+         }
+ 
+         for (final Long flowFileId : beingProcessed) {
+             final MockFlowFile flowFile = originalVersions.get(flowFileId);
+             if (flowFile != null) {
+                 processorQueue.offer(flowFile);
+             }
+         }
+ 
+         rolledback = true;
+         beingProcessed.clear();
+         currentVersions.clear();
+         originalVersions.clear();
+         transferMap.clear();
+         clearTransferState();
+     }
+ 
+     @Override
+     public void transfer(final FlowFile flowFile) {
+         validateState(flowFile);
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("I only accept MockFlowFile");
+         }
+ 
+         beingProcessed.remove(flowFile.getId());
+         processorQueue.offer((MockFlowFile) flowFile);
+     }
+ 
+     @Override
+     public void transfer(final Collection<FlowFile> flowFiles) {
+         for (final FlowFile flowFile : flowFiles) {
+             transfer(flowFile);
+         }
+     }
+ 
+     @Override
+     public void transfer(final FlowFile flowFile, final Relationship relationship) {
+         validateState(flowFile);
+         List<MockFlowFile> list = transferMap.get(relationship);
+         if (list == null) {
+             list = new ArrayList<>();
+             transferMap.put(relationship, list);
+         }
+ 
+         beingProcessed.remove(flowFile.getId());
+         list.add((MockFlowFile) flowFile);
+     }
+ 
+     @Override
+     public void transfer(final Collection<FlowFile> flowFiles, final Relationship relationship) {
+         for (final FlowFile flowFile : flowFiles) {
+             validateState(flowFile);
+         }
+ 
+         List<MockFlowFile> list = transferMap.get(relationship);
+         if (list == null) {
+             list = new ArrayList<>();
+             transferMap.put(relationship, list);
+         }
+ 
+         for (final FlowFile flowFile : flowFiles) {
+             beingProcessed.remove(flowFile.getId());
+             list.add((MockFlowFile) flowFile);
+         }
+     }
+ 
+     @Override
+     public MockFlowFile write(final FlowFile flowFile, final OutputStreamCallback callback) {
+         validateState(flowFile);
+         if (callback == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         try {
+             callback.process(baos);
+         } catch (final IOException e) {
+             throw new ProcessException(e.toString(), e);
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         newFlowFile.setData(baos.toByteArray());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public FlowFile append(final FlowFile flowFile, final OutputStreamCallback callback) {
+         validateState(flowFile);
+         if (callback == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         try {
+             baos.write(mock.getData());
+             callback.process(baos);
+         } catch (final IOException e) {
+             throw new ProcessException(e.toString(), e);
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         newFlowFile.setData(baos.toByteArray());
+         return newFlowFile;
+     }
+ 
+     @Override
+     public MockFlowFile write(final FlowFile flowFile, final StreamCallback callback) {
+         validateState(flowFile);
+         if (callback == null || flowFile == null) {
+             throw new IllegalArgumentException("argument cannot be null");
+         }
+         if (!(flowFile instanceof MockFlowFile)) {
+             throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+         }
+         final MockFlowFile mock = (MockFlowFile) flowFile;
+ 
+         final ByteArrayInputStream in = new ByteArrayInputStream(mock.getData());
+         final ByteArrayOutputStream out = new ByteArrayOutputStream();
+         try {
+             callback.process(in, out);
+         } catch (final IOException e) {
+             throw new ProcessException(e.toString(), e);
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+         newFlowFile.setData(out.toByteArray());
+ 
+         return newFlowFile;
+     }
+ 
+     private byte[] readFully(final InputStream in) throws IOException {
+         final byte[] buffer = new byte[4096];
+         int len;
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         while ((len = in.read(buffer)) >= 0) {
+             baos.write(buffer, 0, len);
+         }
+ 
+         return baos.toByteArray();
+     }
+ 
+     public List<MockFlowFile> getFlowFilesForRelationship(final Relationship relationship) {
+         List<MockFlowFile> list = this.transferMap.get(relationship);
+         if (list == null) {
+             list = new ArrayList<>();
+         }
+ 
+         return list;
+     }
+ 
+     /**
+      * Returns a List of FlowFiles in the order in which they were transferred
+      * to the given relationship
+      *
+      * @param relationship
+      * @return
+      */
+     public List<MockFlowFile> getFlowFilesForRelationship(final String relationship) {
+         final Relationship procRel = new Relationship.Builder().name(relationship).build();
+         return getFlowFilesForRelationship(procRel);
+     }
+ 
+     public MockFlowFile createFlowFile(final File file) throws IOException {
+         return createFlowFile(Files.readAllBytes(file.toPath()));
+     }
+ 
+     public MockFlowFile createFlowFile(final byte[] data) {
+         final MockFlowFile flowFile = create();
+         flowFile.setData(data);
+         return flowFile;
+     }
+ 
+     public MockFlowFile createFlowFile(final byte[] data, final Map<String, String> attrs) {
+         final MockFlowFile ff = createFlowFile(data);
+         ff.putAttributes(attrs);
+         return ff;
+     }
+ 
+     @Override
+     public MockFlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator) {
+         for (final FlowFile flowFile : sources) {
+             validateState(flowFile);
+         }
+         validateState(destination);
+ 
+         final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         try {
+             if (header != null) {
+                 baos.write(header);
+             }
+ 
+             int count = 0;
+             for (final FlowFile flowFile : sources) {
+                 baos.write(((MockFlowFile) flowFile).getData());
+                 if (demarcator != null && ++count != sources.size()) {
+                     baos.write(demarcator);
+                 }
+             }
+ 
+             if (footer != null) {
+                 baos.write(footer);
+             }
+         } catch (final IOException e) {
+             throw new AssertionError("failed to write data to BAOS");
+         }
+ 
+         final MockFlowFile newFlowFile = new MockFlowFile(destination.getId(), destination);
+         newFlowFile.setData(baos.toByteArray());
+         currentVersions.put(newFlowFile.getId(), newFlowFile);
+ 
+         return newFlowFile;
+     }
+ 
+     private void validateState(final FlowFile flowFile) {
+         Objects.requireNonNull(flowFile);
+         final FlowFile currentVersion = currentVersions.get(flowFile.getId());
+         if (currentVersion == null) {
+             throw new FlowFileHandlingException(flowFile + " is not known in this session");
+         }
+ 
+         if (currentVersion != flowFile) {
+             throw new FlowFileHandlingException(flowFile + " is not the most recent version of this flow file within this session");
+         }
+ 
+         for (final List<MockFlowFile> flowFiles : transferMap.values()) {
+             if (flowFiles.contains(flowFile)) {
+                 throw new IllegalStateException(flowFile + " has already been transferred");
+             }
+         }
+     }
+ 
+     /**
+      * Inherits the attributes from the given source flow file into another flow
+      * file. The UUID of the source becomes the parent UUID of this flow file.
+      * If a parent uuid had previously been established it will be replaced by
+      * the uuid of the given source
+      *
+      * @param source the FlowFile from which to copy attributes
+      * @param destination the FlowFile to which to copy attributes
+      */
+     private FlowFile inheritAttributes(final FlowFile source, final FlowFile destination) {
+         if (source == null || destination == null || source == destination) {
+             return destination; //don't need to inherit from ourselves
+         }
+         FlowFile updated = putAllAttributes(destination, source.getAttributes());
+         getProvenanceReporter().fork(source, Collections.singletonList(updated));
+         return updated;
+     }
+ 
+     /**
+      * Inherits the attributes from the given source flow files into the
+      * destination flow file. The UUIDs of the sources becomes the parent UUIDs
+      * of the destination flow file. Only attributes which is common to all
+      * source items is copied into this flow files attributes. Any previously
+      * established parent UUIDs will be replaced by the UUIDs of the sources. It
+      * will capture the uuid of a certain number of source objects and may not
+      * capture all of them. How many it will capture is unspecified.
+      *
+      * @param sources
+      */
+     private FlowFile inheritAttributes(final Collection<FlowFile> sources, final FlowFile destination) {
+         final String uuid = destination.getAttribute(CoreAttributes.UUID.key());
+         final StringBuilder parentUuidBuilder = new StringBuilder();
+         int uuidsCaptured = 0;
+         for (final FlowFile source : sources) {
+             if (source == destination) {
+                 continue; //don't want to capture parent uuid of this.  Something can't be a child of itself
+             }
+             final String sourceUuid = source.getAttribute(CoreAttributes.UUID.key());
+             if (sourceUuid != null && !sourceUuid.trim().isEmpty()) {
+                 uuidsCaptured++;
+                 if (parentUuidBuilder.length() > 0) {
+                     parentUuidBuilder.append(",");
+                 }
+                 parentUuidBuilder.append(sourceUuid);
+             }
+ 
+             if (uuidsCaptured > 100) {
+                 break;
+             }
+         }
+ 
+         FlowFile updated = putAllAttributes(destination, intersectAttributes(sources));
+         getProvenanceReporter().join(sources, updated);
+         return updated;
+     }
+ 
+     /**
+      * Returns the attributes that are common to every flow file given. The key
+      * and value must match exactly.
+      *
+      * @param flowFileList a list of flow files
+      *
+      * @return the common attributes
+      */
+     private static Map<String, String> intersectAttributes(final Collection<FlowFile> flowFileList) {
+         final Map<String, String> result = new HashMap<>();
+         //trivial cases
+         if (flowFileList == null || flowFileList.isEmpty()) {
+             return result;
+         } else if (flowFileList.size() == 1) {
+             result.putAll(flowFileList.iterator().next().getAttributes());
+         }
+ 
+         /*
+          * Start with the first attribute map and only put an entry to the
+          * resultant map if it is common to every map.
+          */
+         final Map<String, String> firstMap = flowFileList.iterator().next().getAttributes();
+ 
+         outer:
+         for (final Map.Entry<String, String> mapEntry : firstMap.entrySet()) {
+             final String key = mapEntry.getKey();
+             final String value = mapEntry.getValue();
+             for (final FlowFile flowFile : flowFileList) {
+                 final Map<String, String> currMap = flowFile.getAttributes();
+                 final String curVal = currMap.get(key);
+                 if (curVal == null || !curVal.equals(value)) {
+                     continue outer;
+                 }
+             }
+             result.put(key, value);
+         }
+ 
+         return result;
+     }
+ 
+     /**
+      * Assert that {@link #commit()} has been called
+      */
+     public void assertCommitted() {
+         Assert.assertTrue("Session was not committed", committed);
+     }
+ 
+     /**
+      * Assert that {@link #commit()} has not been called
+      */
+     public void assertNotCommitted() {
+         Assert.assertFalse("Session was committed", committed);
+     }
+ 
+     /**
+      * Assert that {@link #rollback()} has been called
+      */
+     public void assertRolledBack() {
+         Assert.assertTrue("Session was not rolled back", rolledback);
+     }
+ 
+     /**
+      * Assert that {@link #rollback()} has not been called
+      */
+     public void assertNotRolledBack() {
+         Assert.assertFalse("Session was rolled back", rolledback);
+     }
+ 
+     /**
+      * Assert that the number of FlowFiles transferred to the given relationship
+      * is equal to the given count
+      *
+      * @param relationship
+      * @param count
+      */
+     public void assertTransferCount(final Relationship relationship, final int count) {
+         final int transferCount = getFlowFilesForRelationship(relationship).size();
+         Assert.assertEquals("Expected " + count + " FlowFiles to be transferred to "
+                 + relationship + " but actual transfer count was " + transferCount, count, transferCount);
+     }
+ 
+     /**
+      * Assert that the number of FlowFiles transferred to the given relationship
+      * is equal to the given count
+      *
+      * @param relationship
+      * @param count
+      */
+     public void assertTransferCount(final String relationship, final int count) {
+         assertTransferCount(new Relationship.Builder().name(relationship).build(), count);
+     }
+ 
+     /**
+      * Assert that there are no FlowFiles left on the input queue.
+      */
+     public void assertQueueEmpty() {
+         Assert.assertTrue("FlowFile Queue has " + this.processorQueue.size() + " items", this.processorQueue.isEmpty());
+     }
+ 
+     /**
+      * Assert that at least one FlowFile is on the input queue
+      */
+     public void assertQueueNotEmpty() {
+         Assert.assertFalse("FlowFile Queue is empty", this.processorQueue.isEmpty());
+     }
+ 
+     /**
+      * Asserts that all FlowFiles that were transferred were transferred to the
+      * given relationship
+      *
+      * @param relationship
+      */
+     public void assertAllFlowFilesTransferred(final String relationship) {
+         assertAllFlowFilesTransferred(new Relationship.Builder().name(relationship).build());
+     }
+ 
+     /**
+      * Asserts that all FlowFiles that were transferred were transferred to the
+      * given relationship
+      *
+      * @param relationship
+      */
+     public void assertAllFlowFilesTransferred(final Relationship relationship) {
+         for (final Map.Entry<Relationship, List<MockFlowFile>> entry : transferMap.entrySet()) {
+             final Relationship rel = entry.getKey();
+             final List<MockFlowFile> flowFiles = entry.getValue();
+ 
+             if (!rel.equals(relationship) && flowFiles != null && !flowFiles.isEmpty()) {
+                 Assert.fail("Expected all Transferred FlowFiles to go to " + relationship + " but " + flowFiles.size() + " were routed to " + rel);
+             }
+         }
+     }
+ 
+     /**
+      * Removes all state information about FlowFiles that have been transferred
+      */
+     public void clearTransferState() {
+         this.transferMap.clear();
+     }
+ 
+     /**
+      * Asserts that all FlowFiles that were transferred were transferred to the
+      * given relationship and that the number of FlowFiles transferred is equal
+      * to <code>count</code>
+      *
+      * @param relationship
+      * @param count
+      */
+     public void assertAllFlowFilesTransferred(final Relationship relationship, final int count) {
+         assertAllFlowFilesTransferred(relationship);
+         assertTransferCount(relationship, count);
+     }
+ 
+     /**
+      * Asserts that all FlowFiles that were transferred were transferred to the
+      * given relationship and that the number of FlowFiles transferred is equal
+      * to <code>count</code>
+      *
+      * @param relationship
+      * @param count
+      */
+     public void assertAllFlowFilesTransferred(final String relationship, final int count) {
+         assertAllFlowFilesTransferred(new Relationship.Builder().name(relationship).build(), count);
+     }
+ 
+     /**
+      * Returns the number of FlowFiles that were removed
+      *
+      * @return
+      */
+     public int getRemovedCount() {
+         return removedCount;
+     }
+ 
+     @Override
+     public ProvenanceReporter getProvenanceReporter() {
+         return sharedState.getProvenanceReporter();
+     }
+ 
+     @Override
+     public MockFlowFile penalize(final FlowFile flowFile) {
+         validateState(flowFile);
+         final MockFlowFile mockFlowFile = (MockFlowFile) flowFile;
+         mockFlowFile.setPenalized();
+         return mockFlowFile;
+     }
+ 
+     public byte[] getContentAsByteArray(final MockFlowFile flowFile) {
+         validateState(flowFile);
+         return flowFile.getData();
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
index 0000000,96bef71..13a87de
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
@@@ -1,0 -1,91 +1,72 @@@
+ /*
+  * 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.util;
+ 
 -import java.util.Collections;
 -import java.util.HashSet;
 -import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.atomic.AtomicLong;
+ 
+ import org.apache.nifi.processor.Processor;
 -import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ 
+ public class SharedSessionState {
+ 
+     private final MockFlowFileQueue flowFileQueue;
+     private final ProvenanceReporter provenanceReporter;
++    @SuppressWarnings("unused")
+     private final Processor processor;
+     private final AtomicLong flowFileIdGenerator;
+     private final ConcurrentMap<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
+ 
 -    private volatile Set<Relationship> unavailableRelationships;
+ 
+     public SharedSessionState(final Processor processor, final AtomicLong flowFileIdGenerator) {
+         flowFileQueue = new MockFlowFileQueue();
+         provenanceReporter = new MockProvenanceReporter();
 -        unavailableRelationships = new HashSet<>();
+         this.flowFileIdGenerator = flowFileIdGenerator;
+         this.processor = processor;
+     }
+ 
 -    public Set<Relationship> getAvailableRelationships() {
 -        final Set<Relationship> relationships = new HashSet<>(processor.getRelationships());
 -        relationships.removeAll(unavailableRelationships);
 -        return relationships;
 -    }
 -
 -    public void setUnavailableRelationships(final Set<Relationship> relationships) {
 -        this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
 -    }
 -
 -    public Set<Relationship> getUnavailableRelationships() {
 -        return unavailableRelationships;
 -    }
 -
+     public MockFlowFileQueue getFlowFileQueue() {
+         return flowFileQueue;
+     }
+ 
+     public ProvenanceReporter getProvenanceReporter() {
+         return provenanceReporter;
+     }
+ 
+     public long nextFlowFileId() {
+         return flowFileIdGenerator.getAndIncrement();
+     }
+ 
+     public void adjustCounter(final String name, final long delta) {
+         AtomicLong counter = counterMap.get(name);
+         if (counter == null) {
+             counter = new AtomicLong(0L);
+             AtomicLong existingCounter = counterMap.putIfAbsent(name, counter);
+             if (existingCounter != null) {
+                 counter = existingCounter;
+             }
+         }
+ 
+         counter.addAndGet(delta);
+     }
+ 
+     public Long getCounterValue(final String name) {
+         final AtomicLong counterValue = counterMap.get(name);
+         return (counterValue == null) ? null : counterValue.get();
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 0000000,54b611d..40d5035
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@@ -1,0 -1,492 +1,492 @@@
+ /*
+  * 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.util;
+ 
+ import static java.util.Objects.requireNonNull;
+ 
+ import java.io.ByteArrayInputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.lang.reflect.InvocationTargetException;
+ import java.nio.file.Files;
+ import java.nio.file.Path;
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.atomic.AtomicInteger;
+ import java.util.concurrent.atomic.AtomicLong;
+ 
+ import org.apache.nifi.components.AllowableValue;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.annotation.OnConfigured;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Processor;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.OnAdded;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnShutdown;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.OnUnscheduled;
+ import org.apache.nifi.processor.annotation.TriggerSerially;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ import org.apache.nifi.reporting.InitializationException;
+ 
+ import org.junit.Assert;
+ 
+ public class StandardProcessorTestRunner implements TestRunner {
+ 
+     private final Processor processor;
+     private final MockProcessContext context;
+     private final MockFlowFileQueue flowFileQueue;
+     private final MockSessionFactory sessionFactory;
+     private final SharedSessionState sharedState;
+     private final AtomicLong idGenerator;
+     private final boolean triggerSerially;
+ 
+     private int numThreads = 1;
+     private final AtomicInteger invocations = new AtomicInteger(0);
+ 
+     StandardProcessorTestRunner(final Processor processor) {
+         this.processor = processor;
+         this.idGenerator = new AtomicLong(0L);
+         this.sharedState = new SharedSessionState(processor, idGenerator);
+         this.flowFileQueue = sharedState.getFlowFileQueue();
+         this.sessionFactory = new MockSessionFactory(sharedState);
+         this.context = new MockProcessContext(processor);
+ 
+         final MockProcessorInitializationContext mockInitContext = new MockProcessorInitializationContext(processor, context);
+         processor.initialize(mockInitContext);
+ 
+         try {
+             ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
+         } catch (Exception e) {
+             Assert.fail("Could not invoke methods annotated with @OnAdded annotation due to: " + e);
+         }
+ 
+         triggerSerially = null != processor.getClass().getAnnotation(TriggerSerially.class);
+     }
+ 
+     @Override
+     public void setValidateExpressionUsage(final boolean validate) {
+         context.setValidateExpressionUsage(validate);
+     }
+ 
+     @Override
+     public Processor getProcessor() {
+         return processor;
+     }
+ 
+     @Override
+     public MockProcessContext getProcessContext() {
+         return context;
+     }
+ 
+     @Override
+     public void run() {
+         run(1);
+     }
+ 
+     @Override
+     public void run(int iterations) {
+         run(iterations, true);
+     }
+ 
+     @Override
+     public void run(final int iterations, final boolean stopOnFinish) {
+         run(iterations, stopOnFinish, true);
+     }
+     
+     @Override
+     public void run(final int iterations, final boolean stopOnFinish, final boolean initialize) {
+         if (iterations < 1) {
+             throw new IllegalArgumentException();
+         }
+ 
+         context.assertValid();
+         context.enableExpressionValidation();
+         try {
+             if ( initialize ) {
+                 try {
+                     ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, context);
+                 } catch (Exception e) {
+                     e.printStackTrace();
+                     Assert.fail("Could not invoke methods annotated with @OnScheduled annotation due to: " + e);
+                 }
+             }
+ 
+             final ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+             @SuppressWarnings("unchecked")
+             final Future<Throwable>[] futures = new Future[iterations];
+             for (int i = 0; i < iterations; i++) {
+                 final Future<Throwable> future = executorService.submit(new RunProcessor());
+                 futures[i] = future;
+             }
+ 
+             executorService.shutdown();
+ 
+             int finishedCount = 0;
+             boolean unscheduledRun = false;
+             for (final Future<Throwable> future : futures) {
+                 try {
+                     final Throwable thrown = future.get();   // wait for the result
+                     if (thrown != null) {
+                         throw new AssertionError(thrown);
+                     }
+ 
+                     if (++finishedCount == 1) {
+                         unscheduledRun = true;
+                         try {
+                             ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
+                         } catch (Exception e) {
+                             Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
+                         }
+                     }
+                 } catch (final Exception e) {
+                 }
+             }
+ 
+             if (!unscheduledRun) {
+                 try {
+                     ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
+                 } catch (Exception e) {
+                     Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
+                 }
+             }
+ 
+             if (stopOnFinish) {
+                 try {
+                     ReflectionUtils.invokeMethodsWithAnnotation(OnStopped.class, processor);
+                 } catch (Exception e) {
+                     Assert.fail("Could not invoke methods annotated with @OnStopped annotation due to: " + e);
+                 }
+             }
+         } finally {
+             context.disableExpressionValidation();
+         }
+     }
+ 
+     @Override
+     public void shutdown() {
+         try {
+             ReflectionUtils.invokeMethodsWithAnnotation(OnShutdown.class, processor);
+         } catch (Exception e) {
+             Assert.fail("Could not invoke methods annotated with @OnShutdown annotation due to: " + e);
+         }
+     }
+ 
+     private class RunProcessor implements Callable<Throwable> {
+ 
+         @Override
+         public Throwable call() throws Exception {
+             invocations.incrementAndGet();
+             try {
+                 processor.onTrigger(context, sessionFactory);
+             } catch (final Throwable t) {
+                 return t;
+             }
+ 
+             return null;
+         }
+     }
+ 
+     @Override
+     public ProcessSessionFactory getProcessSessionFactory() {
+         return sessionFactory;
+     }
+ 
+     @Override
+     public void assertAllFlowFilesTransferred(final String relationship) {
+         for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+             session.assertAllFlowFilesTransferred(relationship);
+         }
+     }
+ 
+     @Override
+     public void assertAllFlowFilesTransferred(final Relationship relationship) {
+         for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+             session.assertAllFlowFilesTransferred(relationship);
+         }
+     }
+ 
+     @Override
+     public void assertAllFlowFilesTransferred(final String relationship, final int count) {
+         assertAllFlowFilesTransferred(relationship);
+         assertTransferCount(relationship, count);
+     }
+ 
+     @Override
+     public void assertAllFlowFilesTransferred(final Relationship relationship, final int count) {
+         assertAllFlowFilesTransferred(relationship);
+         assertTransferCount(relationship, count);
+     }
+ 
+     @Override
+     public void assertTransferCount(final Relationship relationship, final int count) {
+         Assert.assertEquals(count, getFlowFilesForRelationship(relationship).size());
+     }
+ 
+     @Override
+     public void assertTransferCount(final String relationship, final int count) {
+         Assert.assertEquals(count, getFlowFilesForRelationship(relationship).size());
+     }
+ 
+     @Override
+     public void assertValid() {
+         context.assertValid();
+     }
+ 
+     @Override
+     public void assertNotValid() {
+         Assert.assertFalse("Processor appears to be valid but expected it to be invalid", context.isValid());
+     }
+ 
+     @Override
+     public boolean isQueueEmpty() {
+         return flowFileQueue.isEmpty();
+     }
+ 
+     @Override
+     public void assertQueueEmpty() {
+         Assert.assertTrue(flowFileQueue.isEmpty());
+     }
+ 
+     @Override
+     public void assertQueueNotEmpty() {
+         Assert.assertFalse(flowFileQueue.isEmpty());
+     }
+ 
+     @Override
+     public void clearTransferState() {
+         for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+             session.clearTransferState();
+         }
+     }
+ 
+     @Override
+     public void enqueue(final FlowFile... flowFiles) {
+         for (final FlowFile flowFile : flowFiles) {
+             flowFileQueue.offer((MockFlowFile) flowFile);
+         }
+     }
+ 
+     @Override
+     public void enqueue(final Path path) throws IOException {
+         enqueue(path, new HashMap<String, String>());
+     }
+ 
+     @Override
+     public void enqueue(final Path path, final Map<String, String> attributes) throws IOException {
+         final Map<String, String> modifiedAttributes = new HashMap<>(attributes);
+         if (!modifiedAttributes.containsKey(CoreAttributes.FILENAME.key())) {
+             modifiedAttributes.put(CoreAttributes.FILENAME.key(), path.toFile().getName());
+         }
+         try (final InputStream in = Files.newInputStream(path)) {
+             enqueue(in, modifiedAttributes);
+         }
+     }
+ 
+     @Override
+     public void enqueue(final byte[] data) {
+         enqueue(data, new HashMap<String, String>());
+     }
+ 
+     @Override
+     public void enqueue(final byte[] data, final Map<String, String> attributes) {
+         enqueue(new ByteArrayInputStream(data), attributes);
+     }
+ 
+     @Override
+     public void enqueue(final InputStream data) {
+         enqueue(data, new HashMap<String, String>());
+     }
+ 
+     @Override
+     public void enqueue(final InputStream data, final Map<String, String> attributes) {
+         final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
+         MockFlowFile flowFile = session.create();
+         flowFile = session.importFrom(data, flowFile);
+         flowFile = session.putAllAttributes(flowFile, attributes);
+         enqueue(flowFile);
+     }
+ 
+     @Override
+     public byte[] getContentAsByteArray(final MockFlowFile flowFile) {
+         return flowFile.getData();
+     }
+ 
+     @Override
+     public List<MockFlowFile> getFlowFilesForRelationship(final String relationship) {
+         final Relationship rel = new Relationship.Builder().name(relationship).build();
+         return getFlowFilesForRelationship(rel);
+     }
+ 
+     @Override
+     public List<MockFlowFile> getFlowFilesForRelationship(final Relationship relationship) {
+         final List<MockFlowFile> flowFiles = new ArrayList<>();
+         for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+             flowFiles.addAll(session.getFlowFilesForRelationship(relationship));
+         }
+ 
+         Collections.sort(flowFiles, new Comparator<MockFlowFile>() {
+             @Override
+             public int compare(final MockFlowFile o1, final MockFlowFile o2) {
+                 return Long.compare(o1.getCreationTime(), o2.getCreationTime());
+             }
+         });
+ 
+         return flowFiles;
+     }
+ 
+     @Override
+     public ProvenanceReporter getProvenanceReporter() {
+         return sharedState.getProvenanceReporter();
+     }
+ 
+     @Override
+     public QueueSize getQueueSize() {
+         return flowFileQueue.size();
+     }
+ 
+     @Override
+     public Long getCounterValue(final String name) {
+         return sharedState.getCounterValue(name);
+     }
+ 
+     @Override
+     public int getRemovedCount() {
+         int count = 0;
+         for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+             count += session.getRemovedCount();
+         }
+ 
+         return count;
+     }
+ 
+     @Override
+     public void setAnnotationData(final String annotationData) {
+         context.setAnnotationData(annotationData);
+     }
+ 
+     @Override
+     public ValidationResult setProperty(final String propertyName, final String propertyValue) {
+         return context.setProperty(propertyName, propertyValue);
+     }
+ 
+     @Override
+     public ValidationResult setProperty(final PropertyDescriptor descriptor, final String value) {
+         return context.setProperty(descriptor, value);
+     }
+ 
+     @Override
+     public ValidationResult setProperty(final PropertyDescriptor descriptor, final AllowableValue value) {
+         return context.setProperty(descriptor, value.getValue());
+     }
+ 
+     @Override
+     public void setThreadCount(final int threadCount) {
+         if (threadCount > 1 && triggerSerially) {
+             Assert.fail("Cannot set thread-count higher than 1 because the processor is triggered serially");
+         }
+ 
+         this.numThreads = threadCount;
+     }
+ 
+     @Override
+     public int getThreadCount() {
+         return numThreads;
+     }
+ 
+     @Override
+     public void setRelationshipAvailable(final Relationship relationship) {
 -        final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
++        final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
+         unavailable.remove(relationship);
 -        sharedState.setUnavailableRelationships(unavailable);
++        context.setUnavailableRelationships(unavailable);
+     }
+ 
+     @Override
+     public void setRelationshipAvailable(final String relationshipName) {
+         setRelationshipAvailable(new Relationship.Builder().name(relationshipName).build());
+     }
+ 
+     @Override
+     public void setRelationshipUnavailable(final Relationship relationship) {
 -        final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
++        final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
+         unavailable.add(relationship);
 -        sharedState.setUnavailableRelationships(unavailable);
++        context.setUnavailableRelationships(unavailable);
+     }
+ 
+     @Override
+     public void setRelationshipUnavailable(final String relationshipName) {
+         setRelationshipUnavailable(new Relationship.Builder().name(relationshipName).build());
+     }
+ 
+     @Override
+     public void addControllerService(final String identifier, final ControllerService service) throws InitializationException {
+         addControllerService(identifier, service, new HashMap<String, String>());
+     }
+ 
+     @Override
+     public void addControllerService(final String identifier, final ControllerService service, final Map<String, String> properties) throws InitializationException {
+         final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier));
+         service.initialize(initContext);
+ 
+         final Map<PropertyDescriptor, String> resolvedProps = new HashMap<>();
+         for (final Map.Entry<String, String> entry : properties.entrySet()) {
+             resolvedProps.put(service.getPropertyDescriptor(entry.getKey()), entry.getValue());
+         }
+ 
+         final MockConfigurationContext configurationContext = new MockConfigurationContext(resolvedProps, context);
+         try {
+             ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, service, configurationContext);
+         } catch (final InvocationTargetException | IllegalAccessException | IllegalArgumentException e) {
+             throw new InitializationException(e);
+         }
+ 
+         context.addControllerService(identifier, service, resolvedProps, null);
+     }
+ 
+     @Override
+     public ControllerService getControllerService(final String identifier) {
+         return context.getControllerService(identifier);
+     }
+ 
+     @Override
+     public <T extends ControllerService> T getControllerService(final String identifier, final Class<T> serviceType) {
+         final ControllerService service = context.getControllerService(identifier);
+         return serviceType.cast(service);
+     }
+ 
+     @Override
+     public boolean removeProperty(PropertyDescriptor descriptor) {
+         return context.removeProperty(descriptor);
+     }
+ 
+ }


[3/8] incubator-nifi git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index 0000000,31e5105..da80546
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@@ -1,0 -1,293 +1,293 @@@
+ /*
+  * 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.processors.standard;
+ 
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedInputStream;
+ import org.apache.nifi.logging.ProcessorLog;
+ import org.apache.nifi.processor.AbstractProcessor;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processors.standard.util.FileInfo;
+ import org.apache.nifi.processors.standard.util.FileTransfer;
+ import org.apache.nifi.processors.standard.util.SFTPTransfer;
+ import org.apache.nifi.util.ObjectHolder;
+ import org.apache.nifi.util.StopWatch;
+ 
+ import java.io.File;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+ 
+ /**
+  * Base class for PutFTP & PutSFTP
+  * @param <T>
+  */
+ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractProcessor {
+ 
+     public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("FlowFiles that are successfully sent will be routed to success").build();
+     public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles that failed to send to the remote system; failure is usually looped back to this processor").build();
+     public static final Relationship REL_REJECT = new Relationship.Builder().name("reject").description("FlowFiles that were rejected by the destination system").build();
+ 
+     private final Set<Relationship> relationships;
+ 
+     public PutFileTransfer() {
+         super();
+         final Set<Relationship> relationships = new HashSet<>();
+         relationships.add(REL_SUCCESS);
+         relationships.add(REL_FAILURE);
+         relationships.add(REL_REJECT);
+         this.relationships = Collections.unmodifiableSet(relationships);
+     }
+ 
+     @Override
+     public Set<Relationship> getRelationships() {
+         return relationships;
+     }
+ 
+     protected abstract T getFileTransfer(final ProcessContext context);
+ 
+     protected void beforePut(final FlowFile flowFile, final ProcessContext context, final T transfer) throws IOException {
+ 
+     }
+ 
+     protected void afterPut(final FlowFile flowFile, final ProcessContext context, final T transfer) throws IOException {
+ 
+     }
+ 
+     @Override
+     public void onTrigger(final ProcessContext context, final ProcessSession session) {
+         FlowFile flowFile = session.get();
+         if (flowFile == null) {
+             return;
+         }
+ 
+         final ProcessorLog logger = getLogger();
+         final String hostname = context.getProperty(FileTransfer.HOSTNAME).evaluateAttributeExpressions(flowFile).getValue();
+ 
+         final int maxNumberOfFiles = context.getProperty(FileTransfer.BATCH_SIZE).asInteger();
+         int fileCount = 0;
+         try (final T transfer = getFileTransfer(context)) {
+             do {
+                 final String rootPath = context.getProperty(FileTransfer.REMOTE_PATH).evaluateAttributeExpressions(flowFile).getValue();
+                 final String workingDirPath;
+                 if (rootPath == null) {
+                     workingDirPath = null;
+                 } else {
+                     File workingDirectory = new File(rootPath);
+                     if (!workingDirectory.getPath().startsWith("/") && !workingDirectory.getPath().startsWith("\\")) {
+                         workingDirectory = new File(transfer.getHomeDirectory(flowFile), workingDirectory.getPath());
+                     }
+                     workingDirPath = workingDirectory.getPath().replace("\\", "/");
+                 }
+ 
+                 final boolean rejectZeroByteFiles = context.getProperty(FileTransfer.REJECT_ZERO_BYTE).asBoolean();
+                 final ConflictResult conflictResult = identifyAndResolveConflictFile(context.getProperty(FileTransfer.CONFLICT_RESOLUTION).getValue(),
+                         transfer, workingDirPath, flowFile, rejectZeroByteFiles, logger);
+ 
+                 if (conflictResult.isTransfer()) {
+                     final StopWatch stopWatch = new StopWatch();
+                     stopWatch.start();
+ 
+                     beforePut(flowFile, context, transfer);
+                     final FlowFile flowFileToTransfer = flowFile;
+                     final ObjectHolder<String> fullPathRef = new ObjectHolder<>(null);
+                     session.read(flowFile, new InputStreamCallback() {
+                         @Override
+                         public void process(final InputStream in) throws IOException {
+                             try (final InputStream bufferedIn = new BufferedInputStream(in)) {
+                                 if (workingDirPath != null && context.getProperty(SFTPTransfer.CREATE_DIRECTORY).asBoolean()) {
+                                     transfer.ensureDirectoryExists(flowFileToTransfer, new File(workingDirPath));
+                                 }
+ 
+                                 fullPathRef.set(transfer.put(flowFileToTransfer, workingDirPath, conflictResult.getFileName(), bufferedIn));
+                             }
+                         }
+                     });
+                     afterPut(flowFile, context, transfer);
+ 
+                     stopWatch.stop();
+                     final String dataRate = stopWatch.calculateDataRate(flowFile.getSize());
+                     final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+                     logger.info("Successfully transfered {} to {} on remote host {} in {} milliseconds at a rate of {}",
+                             new Object[]{flowFile, fullPathRef.get(), hostname, millis, dataRate});
+ 
+                     String fullPathWithSlash = fullPathRef.get();
+                     if (!fullPathWithSlash.startsWith("/")) {
+                         fullPathWithSlash = "/" + fullPathWithSlash;
+                     }
+                     final String destinationUri = transfer.getProtocolName() + "://" + hostname + fullPathWithSlash;
+                     session.getProvenanceReporter().send(flowFile, destinationUri, millis);
+                 }
+ 
+                 if (conflictResult.isPenalize()) {
+                     flowFile = session.penalize(flowFile);
+                 }
+ 
+                 session.transfer(flowFile, conflictResult.getRelationship());
+                 session.commit();
 -            } while (isScheduled() && (getRelationships().size() == session.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
++            } while (isScheduled() && (getRelationships().size() == context.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
+         } catch (final IOException e) {
+             context.yield();
+             logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e});
+             flowFile = session.penalize(flowFile);
+             session.transfer(flowFile, REL_FAILURE);
+         } catch (final FlowFileAccessException e) {
+             context.yield();
+             logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e.getCause()});
+             flowFile = session.penalize(flowFile);
+             session.transfer(flowFile, REL_FAILURE);
+         } catch (final ProcessException e) {
+             context.yield();
+             logger.error("Unable to transfer {} to remote host {} due to {}: {}; routing to failure", new Object[]{flowFile, hostname, e, e.getCause()});
+             flowFile = session.penalize(flowFile);
+             session.transfer(flowFile, REL_FAILURE);
+         }
+     }
+ 
+     //Attempts to identify naming or content issues with files before they are transferred.
+     private ConflictResult identifyAndResolveConflictFile(final String conflictResolutionType, final T transfer, final String path, final FlowFile flowFile, final boolean rejectZeroByteFiles, final ProcessorLog logger) throws IOException {
+         Relationship destinationRelationship = REL_SUCCESS;
+         String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+         boolean transferFile = true;
+         boolean penalizeFile = false;
+ 
+         //First, check if the file is empty
+         //Reject files that are zero bytes or less
+         if (rejectZeroByteFiles) {
+             final long sizeInBytes = flowFile.getSize();
+             if (sizeInBytes == 0) {
+                 logger.warn("Rejecting {} because it is zero bytes", new Object[]{flowFile});
+                 return new ConflictResult(REL_REJECT, false, fileName, true);
+             }
+         }
+ 
+         //Second, check if the user doesn't care about detecting naming conflicts ahead of time
+         if (conflictResolutionType.equalsIgnoreCase(FileTransfer.CONFLICT_RESOLUTION_NONE)) {
+             return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+         }
+ 
+         final FileInfo remoteFileInfo = transfer.getRemoteFileInfo(flowFile, path, fileName);
+         if (remoteFileInfo == null) {
+             return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+         }
+ 
+         if (remoteFileInfo.isDirectory()) {
+             logger.info("Resolving conflict by rejecting {} due to conflicting filename with a directory or file already on remote server", new Object[]{flowFile});
+             return new ConflictResult(REL_REJECT, false, fileName, false);
+         }
+ 
+         logger.info("Discovered a filename conflict on the remote server for {} so handling using configured Conflict Resolution of {}",
+                 new Object[]{flowFile, conflictResolutionType});
+ 
+         switch (conflictResolutionType.toUpperCase()) {
+             case FileTransfer.CONFLICT_RESOLUTION_REJECT:
+                 destinationRelationship = REL_REJECT;
+                 transferFile = false;
+                 penalizeFile = false;
+                 logger.info("Resolving conflict by rejecting {} due to conflicting filename with a directory or file already on remote server", new Object[]{flowFile});
+                 break;
+             case FileTransfer.CONFLICT_RESOLUTION_REPLACE:
+                 transfer.deleteFile(path, fileName);
+                 destinationRelationship = REL_SUCCESS;
+                 transferFile = true;
+                 penalizeFile = false;
+                 logger.info("Resolving filename conflict for {} with remote server by deleting remote file and replacing with flow file", new Object[]{flowFile});
+                 break;
+             case FileTransfer.CONFLICT_RESOLUTION_RENAME:
+                 boolean uniqueNameGenerated = false;
+                 for (int i = 1; i < 100 && !uniqueNameGenerated; i++) {
+                     String possibleFileName = i + "." + fileName;
+ 
+                     final FileInfo renamedFileInfo = transfer.getRemoteFileInfo(flowFile, path, possibleFileName);
+                     uniqueNameGenerated = (renamedFileInfo == null);
+                     if (uniqueNameGenerated) {
+                         fileName = possibleFileName;
+                         logger.info("Attempting to resolve filename conflict for {} on the remote server by using a newly generated filename of: {}", new Object[]{flowFile, fileName});
+                         destinationRelationship = REL_SUCCESS;
+                         transferFile = true;
+                         penalizeFile = false;
+                         break;
+                     }
+                 }
+                 if (!uniqueNameGenerated) {
+                     destinationRelationship = REL_REJECT;
+                     transferFile = false;
+                     penalizeFile = false;
+                     logger.info("Could not determine a unique name after 99 attempts for.  Switching resolution mode to REJECT for " + flowFile);
+                 }
+                 break;
+             case FileTransfer.CONFLICT_RESOLUTION_IGNORE:
+                 destinationRelationship = REL_SUCCESS;
+                 transferFile = false;
+                 penalizeFile = false;
+                 logger.info("Resolving conflict for {}  by not transferring file and and still considering the process a success.", new Object[]{flowFile});
+                 break;
+             case FileTransfer.CONFLICT_RESOLUTION_FAIL:
+                 destinationRelationship = REL_FAILURE;
+                 transferFile = false;
+                 penalizeFile = true;
+                 logger.info("Resolved filename conflict for {} as configured by routing to FAILURE relationship.", new Object[]{flowFile});
+             default:
+                 break;
+         }
+ 
+         return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+     }
+ 
+     /**
+      * static inner class to hold conflict data
+      */
+     private static class ConflictResult {
+ 
+         final Relationship relationship;
+         final boolean transferFile;
+         final String newFileName;
+         final boolean penalizeFile;
+ 
+         public ConflictResult(final Relationship relationship, final boolean transferFileVal, final String newFileNameVal, final boolean penalizeFileVal) {
+             this.relationship = relationship;
+             this.transferFile = transferFileVal;
+             this.newFileName = newFileNameVal;
+             this.penalizeFile = penalizeFileVal;
+         }
+ 
+         public boolean isTransfer() {
+             return transferFile;
+         }
+ 
+         public boolean isPenalize() {
+             return penalizeFile;
+         }
+ 
+         public String getFileName() {
+             return newFileName;
+         }
+ 
+         public Relationship getRelationship() {
+             return relationship;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
index 0000000,cae61f0..1cf5f1f
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
@@@ -1,0 -1,320 +1,325 @@@
+ /*
+  * 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.processors.standard.servlets;
+ 
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.security.cert.X509Certificate;
+ import java.util.Enumeration;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.regex.Pattern;
+ import java.util.zip.GZIPInputStream;
+ 
+ import javax.servlet.ServletConfig;
+ import javax.servlet.ServletContext;
+ import javax.servlet.ServletException;
+ import javax.servlet.http.HttpServlet;
+ import javax.servlet.http.HttpServletRequest;
+ import javax.servlet.http.HttpServletResponse;
+ import javax.ws.rs.Path;
+ import javax.ws.rs.core.MediaType;
+ 
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedOutputStream;
+ import org.apache.nifi.stream.io.StreamThrottler;
+ import org.apache.nifi.logging.ProcessorLog;
++import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processors.standard.ListenHTTP;
+ import org.apache.nifi.processors.standard.ListenHTTP.FlowFileEntryTimeWrapper;
+ import org.apache.nifi.util.FlowFileUnpackager;
+ import org.apache.nifi.util.FlowFileUnpackagerV1;
+ import org.apache.nifi.util.FlowFileUnpackagerV2;
+ import org.apache.nifi.util.FlowFileUnpackagerV3;
+ import org.apache.commons.io.IOUtils;
+ import org.apache.commons.lang3.StringUtils;
+ 
+ @Path(ListenHTTP.URI)
+ public class ListenHTTPServlet extends HttpServlet {
+ 
+     private static final long serialVersionUID = 5329940480987723163L;
+ 
+     public static final String FLOWFILE_CONFIRMATION_HEADER = "x-prefer-acknowledge-uri";
+     public static final String LOCATION_HEADER_NAME = "Location";
+     public static final String DEFAULT_FOUND_SUBJECT = "none";
+     public static final String APPLICATION_FLOW_FILE_V1 = "application/flowfile";
+     public static final String APPLICATION_FLOW_FILE_V2 = "application/flowfile-v2";
+     public static final String APPLICATION_FLOW_FILE_V3 = "application/flowfile-v3";
+     public static final String LOCATION_URI_INTENT_NAME = "x-location-uri-intent";
+     public static final String LOCATION_URI_INTENT_VALUE = "flowfile-hold";
+     public static final int FILES_BEFORE_CHECKING_DESTINATION_SPACE = 5;
+     public static final String ACCEPT_HEADER_NAME = "Accept";
+     public static final String ACCEPT_HEADER_VALUE = APPLICATION_FLOW_FILE_V3 + "," + APPLICATION_FLOW_FILE_V2 + "," + APPLICATION_FLOW_FILE_V1 + ",*/*;q=0.8";
+     public static final String ACCEPT_ENCODING_NAME = "Accept-Encoding";
+     public static final String ACCEPT_ENCODING_VALUE = "gzip";
+     public static final String GZIPPED_HEADER = "flowfile-gzipped";
+     public static final String PROTOCOL_VERSION_HEADER = "x-nifi-transfer-protocol-version";
+     public static final String PROTOCOL_VERSION = "3";
+ 
+     private final AtomicLong filesReceived = new AtomicLong(0L);
+     private final AtomicBoolean spaceAvailable = new AtomicBoolean(true);
+ 
+     private ProcessorLog logger;
+     private AtomicReference<ProcessSessionFactory> sessionFactoryHolder;
++    private volatile ProcessContext processContext;
+     private Pattern authorizedPattern;
+     private Pattern headerPattern;
+     private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
+     private StreamThrottler streamThrottler;
+ 
+     /**
+      *
+      * @param config
+      * @throws ServletException
+      */
+     @SuppressWarnings("unchecked")
+     @Override
+     public void init(final ServletConfig config) throws ServletException {
+         final ServletContext context = config.getServletContext();
+         this.logger = (ProcessorLog) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_LOGGER);
+         this.sessionFactoryHolder = (AtomicReference<ProcessSessionFactory>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER);
++        this.processContext = (ProcessContext) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER);
+         this.authorizedPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN);
+         this.headerPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_HEADER_PATTERN);
+         this.flowFileMap = (ConcurrentMap<String, FlowFileEntryTimeWrapper>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_FLOWFILE_MAP);
+         this.streamThrottler = (StreamThrottler) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_STREAM_THROTTLER);
+     }
+ 
+     @Override
+     protected void doHead(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+         response.addHeader(ACCEPT_ENCODING_NAME, ACCEPT_ENCODING_VALUE);
+         response.addHeader(ACCEPT_HEADER_NAME, ACCEPT_HEADER_VALUE);
+         response.addHeader(PROTOCOL_VERSION_HEADER, PROTOCOL_VERSION);
+     }
+ 
+     @Override
+     protected void doPost(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
++        final ProcessContext context = processContext;
++        
+         ProcessSessionFactory sessionFactory;
+         do {
+             sessionFactory = sessionFactoryHolder.get();
+             if (sessionFactory == null) {
+                 try {
+                     Thread.sleep(10);
+                 } catch (final InterruptedException e) {
+                 }
+             }
+         } while (sessionFactory == null);
+ 
+         final ProcessSession session = sessionFactory.createSession();
+         FlowFile flowFile = null;
+         String holdUuid = null;
+         String foundSubject = null;
+         try {
+             final long n = filesReceived.getAndIncrement() % FILES_BEFORE_CHECKING_DESTINATION_SPACE;
+             if (n == 0 || !spaceAvailable.get()) {
 -                if (session.getAvailableRelationships().isEmpty()) {
++                if (context.getAvailableRelationships().isEmpty()) {
+                     spaceAvailable.set(false);
+                     if (logger.isDebugEnabled()) {
+                         logger.debug("Received request from " + request.getRemoteHost() + " but no space available; Indicating Service Unavailable");
+                     }
+                     response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
+                     return;
+                 } else {
+                     spaceAvailable.set(true);
+                 }
+             }
+             response.setHeader("Content-Type", MediaType.TEXT_PLAIN);
+ 
+             final boolean contentGzipped = Boolean.parseBoolean(request.getHeader(GZIPPED_HEADER));
+ 
+             final X509Certificate[] certs = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
+             foundSubject = DEFAULT_FOUND_SUBJECT;
+             if (certs != null && certs.length > 0) {
+                 for (final X509Certificate cert : certs) {
+                     foundSubject = cert.getSubjectDN().getName();
+                     if (authorizedPattern.matcher(foundSubject).matches()) {
+                         break;
+                     } else {
+                         logger.warn("Rejecting transfer attempt from " + foundSubject + " because the DN is not authorized, host=" + request.getRemoteHost());
+                         response.sendError(HttpServletResponse.SC_FORBIDDEN, "not allowed based on dn");
+                         return;
+                     }
+                 }
+             }
+ 
+             final String destinationVersion = request.getHeader(PROTOCOL_VERSION_HEADER);
+             Integer protocolVersion = null;
+             if (destinationVersion != null) {
+                 try {
+                     protocolVersion = Integer.valueOf(destinationVersion);
+                 } catch (final NumberFormatException e) {
+                     // Value was invalid. Treat as if the header were missing.
+                 }
+             }
+ 
+             final boolean destinationIsLegacyNiFi = (protocolVersion == null);
+             final boolean createHold = Boolean.parseBoolean(request.getHeader(FLOWFILE_CONFIRMATION_HEADER));
+             final String contentType = request.getContentType();
+ 
+             final InputStream unthrottled = contentGzipped ? new GZIPInputStream(request.getInputStream()) : request.getInputStream();
+ 
+             final InputStream in = (streamThrottler == null) ? unthrottled : streamThrottler.newThrottledInputStream(unthrottled);
+ 
+             if (logger.isDebugEnabled()) {
+                 logger.debug("Received request from " + request.getRemoteHost() + ", createHold=" + createHold + ", content-type=" + contentType + ", gzip=" + contentGzipped);
+             }
+ 
+             final AtomicBoolean hasMoreData = new AtomicBoolean(false);
+             final FlowFileUnpackager unpackager;
+             if (APPLICATION_FLOW_FILE_V3.equals(contentType)) {
+                 unpackager = new FlowFileUnpackagerV3();
+             } else if (APPLICATION_FLOW_FILE_V2.equals(contentType)) {
+                 unpackager = new FlowFileUnpackagerV2();
+             } else if (APPLICATION_FLOW_FILE_V1.equals(contentType)) {
+                 unpackager = new FlowFileUnpackagerV1();
+             } else {
+                 unpackager = null;
+             }
+ 
+             final Set<FlowFile> flowFileSet = new HashSet<>();
+ 
+             do {
+                 final long startNanos = System.nanoTime();
+                 final Map<String, String> attributes = new HashMap<>();
+                 flowFile = session.create();
+                 flowFile = session.write(flowFile, new OutputStreamCallback() {
+                     @Override
+                     public void process(final OutputStream rawOut) throws IOException {
+                         try (final BufferedOutputStream bos = new BufferedOutputStream(rawOut, 65536)) {
+                             if (unpackager == null) {
+                                 IOUtils.copy(in, bos);
+                                 hasMoreData.set(false);
+                             } else {
+                                 attributes.putAll(unpackager.unpackageFlowFile(in, bos));
+ 
+                                 if (destinationIsLegacyNiFi) {
+                                     if (attributes.containsKey("nf.file.name")) {
+                                         // for backward compatibility with old nifi...
+                                         attributes.put(CoreAttributes.FILENAME.key(), attributes.remove("nf.file.name"));
+                                     }
+ 
+                                     if (attributes.containsKey("nf.file.path")) {
+                                         attributes.put(CoreAttributes.PATH.key(), attributes.remove("nf.file.path"));
+                                     }
+                                 }
+ 
+                                 // remove deprecated FlowFile attribute that was used in older versions of NiFi
+                                 attributes.remove("parent.uuid");
+ 
+                                 hasMoreData.set(unpackager.hasMoreData());
+                             }
+                         }
+                     }
+                 });
+ 
+                 final long transferNanos = System.nanoTime() - startNanos;
+                 final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+ 
+                 // put metadata on flowfile
+                 final String nameVal = request.getHeader(CoreAttributes.FILENAME.key());
+                 if (StringUtils.isNotBlank(nameVal)) {
+                     attributes.put(CoreAttributes.FILENAME.key(), nameVal);
+                 }
+                 
+                 // put arbitrary headers on flow file
+                 for(Enumeration<String> headerEnum = request.getHeaderNames(); 
+                 		headerEnum.hasMoreElements(); ) {
+                 	String headerName = headerEnum.nextElement();
+                 	if (headerPattern != null && headerPattern.matcher(headerName).matches()) {
+ 	                	String headerValue = request.getHeader(headerName);
+ 	                	attributes.put(headerName, headerValue);
+ 	                }
+                 }
+ 
+                 String sourceSystemFlowFileIdentifier = attributes.get(CoreAttributes.UUID.key());
+                 if (sourceSystemFlowFileIdentifier != null) {
+                     sourceSystemFlowFileIdentifier = "urn:nifi:" + sourceSystemFlowFileIdentifier;
+ 
+                     // If we receveied a UUID, we want to give the FlowFile a new UUID and register the sending system's
+                     // identifier as the SourceSystemFlowFileIdentifier field in the Provenance RECEIVE event
+                     attributes.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+                 }
+ 
+                 flowFile = session.putAllAttributes(flowFile, attributes);
+                 session.getProvenanceReporter().receive(flowFile, request.getRequestURL().toString(), sourceSystemFlowFileIdentifier, "Remote DN=" + foundSubject, transferMillis);
+                 flowFile = session.putAttribute(flowFile, "restlistener.remote.user.dn", foundSubject);
+                 flowFileSet.add(flowFile);
+ 
+                 if (holdUuid == null) {
+                     holdUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+                 }
+             } while (hasMoreData.get());
+ 
+             if (createHold) {
+                 String uuid = (holdUuid == null) ? UUID.randomUUID().toString() : holdUuid;
+ 
+                 if (flowFileMap.containsKey(uuid)) {
+                     uuid = UUID.randomUUID().toString();
+                 }
+ 
+                 final FlowFileEntryTimeWrapper wrapper = new FlowFileEntryTimeWrapper(session, flowFileSet, System.currentTimeMillis());
+                 FlowFileEntryTimeWrapper previousWrapper;
+                 do {
+                     previousWrapper = flowFileMap.putIfAbsent(uuid, wrapper);
+                     if (previousWrapper != null) {
+                         uuid = UUID.randomUUID().toString();
+                     }
+                 } while (previousWrapper != null);
+ 
+                 response.setStatus(HttpServletResponse.SC_SEE_OTHER);
+                 final String ackUri = ListenHTTP.URI + "/holds/" + uuid;
+                 response.addHeader(LOCATION_HEADER_NAME, ackUri);
+                 response.addHeader(LOCATION_URI_INTENT_NAME, LOCATION_URI_INTENT_VALUE);
+                 response.getOutputStream().write(ackUri.getBytes("UTF-8"));
+                 if (logger.isDebugEnabled()) {
+                     logger.debug("Ingested {} from Remote Host: [{}] Port [{}] SubjectDN [{}]; placed hold on these {} files with ID {}",
+                             new Object[]{flowFileSet, request.getRemoteHost(), request.getRemotePort(), foundSubject, flowFileSet.size(), uuid});
+                 }
+             } else {
+                 response.setStatus(HttpServletResponse.SC_OK);
+                 logger.info("Received from Remote Host: [{}] Port [{}] SubjectDN [{}]; transferring to 'success' {}",
+                         new Object[]{request.getRemoteHost(), request.getRemotePort(), foundSubject, flowFile});
+ 
+                 session.transfer(flowFileSet, ListenHTTP.RELATIONSHIP_SUCCESS);
+                 session.commit();
+             }
+         } catch (final Throwable t) {
+             session.rollback();
+             if (flowFile == null) {
+                 logger.error("Unable to receive file from Remote Host: [{}] SubjectDN [{}] due to {}", new Object[]{request.getRemoteHost(), foundSubject, t});
+             } else {
+                 logger.error("Unable to receive file {} from Remote Host: [{}] SubjectDN [{}] due to {}", new Object[]{flowFile, request.getRemoteHost(), foundSubject, t});
+             }
+             response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, t.toString());
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
index 0000000,a6402e4..ab4c978
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
@@@ -1,0 -1,138 +1,139 @@@
+ /*
+  * 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.processors.standard;
+ 
+ import org.apache.nifi.processors.standard.DistributeLoad;
+ import org.apache.nifi.util.TestRunner;
+ import org.apache.nifi.util.TestRunners;
+ 
+ import org.junit.Assert;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ public class TestDistributeLoad {
+ 
+     @BeforeClass
+     public static void before() {
+         System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+         System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+         System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DistributeLoad", "debug");
+     }
+ 
+     @Test
+     public void testDefaultRoundRobin() {
+         final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+         testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+ 
+         for (int i = 0; i < 101; i++) {
+             testRunner.enqueue(new byte[0]);
+         }
+ 
+         testRunner.run(101);
+         testRunner.assertTransferCount("1", 2);
+         for (int i = 2; i <= 100; i++) {
+             testRunner.assertTransferCount(String.valueOf(i), 1);
+         }
+     }
+ 
+     @Test
+     public void testWeightedRoundRobin() {
+         final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+         testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+ 
+         testRunner.setProperty("1", "5");
+         testRunner.setProperty("2", "3");
+ 
+         for (int i = 0; i < 106; i++) {
+             testRunner.enqueue(new byte[0]);
+         }
+ 
+         testRunner.run(108);
+         testRunner.assertTransferCount("1", 5);
+         testRunner.assertTransferCount("2", 3);
+         for (int i = 3; i <= 100; i++) {
+             testRunner.assertTransferCount(String.valueOf(i), 1);
+         }
+     }
+ 
+     @Test
+     public void testValidationOnAddedProperties() {
+         final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+         testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+ 
+         testRunner.setProperty("1", "5");
+ 
+         try {
+             testRunner.setProperty("1", "0");
+             Assert.fail("Allows property '1' to be set to '0'");
+         } catch (final AssertionError e) {
+             // expected behavior
+         }
+ 
+         try {
+             testRunner.setProperty("1", "-1");
+             Assert.fail("Allows property '1' to be set to '-1'");
+         } catch (final AssertionError e) {
+             // expected behavior
+         }
+ 
+         testRunner.setProperty("1", "101");
+         testRunner.setProperty("100", "5");
+ 
+         try {
+             testRunner.setProperty("101", "5");
+             Assert.fail("Allows property '101' to be set to '5'");
+         } catch (final AssertionError e) {
+             // expected behavior
+         }
+ 
+         try {
+             testRunner.setProperty("0", "5");
+             Assert.fail("Allows property '0' to be set to '5'");
+         } catch (final AssertionError e) {
+             // expected behavior
+         }
+ 
+         try {
+             testRunner.setProperty("-1", "5");
+             Assert.fail("Allows property '-1' to be set to '5'");
+         } catch (final AssertionError e) {
+             // expected behavior
+         }
+     }
+ 
+     @Test
+     public void testNextAvailable() {
+         final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+ 
+         testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS.getName(), "100");
+         testRunner.setProperty(DistributeLoad.DISTRIBUTION_STRATEGY.getName(), DistributeLoad.STRATEGY_NEXT_AVAILABLE);
+ 
+         for (int i = 0; i < 99; i++) {
+             testRunner.enqueue(new byte[0]);
+         }
+ 
+         testRunner.setRelationshipUnavailable("50");
+ 
+         testRunner.run(101);
+         testRunner.assertQueueEmpty();
+ 
+         for (int i = 1; i <= 100; i++) {
++            System.out.println(i);
+             testRunner.assertTransferCount(String.valueOf(i), (i == 50) ? 0 : 1);
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
index 0000000,9e04439..7fa183f
mode 000000,100644..100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
@@@ -1,0 -1,124 +1,132 @@@
+ /*
+  * 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.processor;
+ 
+ import java.util.Map;
++import java.util.Set;
+ 
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ 
+ /**
+  * <p>
+  * Provides a bridge between a Processor and the NiFi Framework
+  * </p>
+  *
+  * <p>
+  * <b>Note: </b>Implementations of this interface are NOT necessarily
+  * thread-safe.
+  * </p>
+  */
+ public interface ProcessContext {
+ 
+     /**
+      * Retrieves the current value set for the given descriptor, if a value is
+      * set - else uses the descriptor to determine the appropriate default value
+      *
+      * @param descriptor
+      * @return
+      */
+     PropertyValue getProperty(PropertyDescriptor descriptor);
+ 
+     /**
+      * Retrieves the current value set for the given descriptor, if a value is
+      * set - else uses the descriptor to determine the appropriate default value
+      *
+      * @param propertyName
+      * @return
+      */
+     PropertyValue getProperty(String propertyName);
+ 
+     /**
+      * Creates and returns a {@link PropertyValue} object that can be used for
+      * evaluating the value of the given String
+      *
+      * @param rawValue
+      * @return
+      */
+     PropertyValue newPropertyValue(String rawValue);
+ 
+     /**
+      * <p>
+      * Causes the Processor not to be scheduled for some pre-configured amount
+      * of time. The duration of time for which the processor will not be
+      * scheduled is configured in the same manner as the processor's scheduling
+      * period.
+      * </p>
+      *
+      * <p>
+      * <b>Note: </b>This is NOT a blocking call and does not suspend execution
+      * of the current thread.
+      * </p>
+      */
+     void yield();
+ 
+     /**
+      * @return the maximum number of threads that may be executing this
+      * processor's code at any given time
+      */
+     int getMaxConcurrentTasks();
+ 
+     /**
+      * @return the annotation data configured for this processor
+      */
+     String getAnnotationData();
+ 
+     /**
+      * Returns a Map of all PropertyDescriptors to their configured values. This
+      * Map may or may not be modifiable, but modifying its values will not
+      * change the values of the processor's properties
+      *
+      * @return
+      */
+     Map<PropertyDescriptor, String> getProperties();
+ 
+     /**
+      * Encrypts the given value using the password provided in the NiFi
+      * Properties
+      *
+      * @param unencrypted
+      * @return
+      */
+     String encrypt(String unencrypted);
+ 
+     /**
+      * Decrypts the given value using the password provided in the NiFi
+      * Properties
+      *
+      * @param encrypted
+      * @return
+      */
+     String decrypt(String encrypted);
+ 
+     /**
+      * Provides a {@code ControllerServiceLookup} that can be used to obtain a
+      * Controller Service
+      *
+      * @return
+      */
+     ControllerServiceLookup getControllerServiceLookup();
++    
++    /**
++     * @return the set of all relationships for which space is available to
++     * receive new objects
++     */
++    Set<Relationship> getAvailableRelationships();
++
+ }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
index 0000000,09d1bd2..d3de916
mode 000000,100644..100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
@@@ -1,0 -1,719 +1,713 @@@
+ /*
+  * 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.processor;
+ 
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Path;
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.regex.Pattern;
+ 
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.MissingFlowFileException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ 
+ /**
+  * <p>
+  * A process session encompasses all the behaviors a processor can perform to
+  * obtain, clone, read, modify remove FlowFiles in an atomic unit. A process
+  * session is always tied to a single processor at any one time and ensures no
+  * FlowFile can ever be accessed by any more than one processor at a given time.
+  * The session also ensures that all FlowFiles are always accounted for. The
+  * creator of a ProcessSession is always required to manage the session.</p>
+  *
+  * <p>
+  * A session is not considered thread safe. The session supports a unit of work
+  * that is either committed or rolled back</p>
+  *
+  * <p>
+  * As noted on specific methods and for specific exceptions automated rollback
+  * will occur to ensure consistency of the repository. However, several
+  * situations can result in exceptions yet not cause automated rollback. In
+  * these cases the consistency of the repository will be retained but callers
+  * will be able to indicate whether it should result in rollback or continue on
+  * toward a commit.</p>
+  *
+  * <p>
+  * A process session instance may be used continuously. That is, after each
+  * commit or rollback, the session can be used again.</p>
+  *
+  * @author unattributed
+  */
+ public interface ProcessSession {
+ 
+     /**
+      * <p>
+      * Commits the current session ensuring all operations against FlowFiles
+      * within this session are atomically persisted. All FlowFiles operated on
+      * within this session must be accounted for by transfer or removal or the
+      * commit will fail.</p>
+      *
+      * <p>
+      * As soon as the commit completes the session is again ready to be used</p>
+      *
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session.
+      * @throws FlowFileHandlingException if not all FlowFiles acted upon within
+      * this session are accounted for by user code such that they have a
+      * transfer identified or where marked for removal. Automated rollback
+      * occurs.
+      * @throws ProcessException if some general fault occurs while persisting
+      * the session. Initiates automatic rollback. The root cause can be obtained
+      * via <code>Exception.getCause()</code>
+      */
+     void commit();
+ 
+     /**
+      * Reverts any changes made during this session. All FlowFiles are restored
+      * back to their initial session state and back to their original queues. If
+      * this session is already committed or rolled back then no changes will
+      * occur. This method can be called any number of times. Calling this method
+      * is identical to calling {@link #rollback(boolean)} passing
+      * <code>false</code> as the parameter.
+      */
+     void rollback();
+ 
+     /**
+      * Reverts any changes made during this session. All FlowFiles are restored
+      * back to their initial session state and back to their original queues,
+      * after optionally being penalized. If this session is already committed or
+      * rolled back then no changes will occur. This method can be called any
+      * number of times.
+      *
+      * @param penalize whether or not the FlowFiles that are being restored back
+      * to their queues should be penalized
+      */
+     void rollback(boolean penalize);
+ 
+     /**
+      * Adjusts counter data for the given counter name and takes care of
+      * registering the counter if not already present. The adjustment occurs
+      * only if and when the ProcessSession is committed.
+      *
+      * @param name the name of the counter
+      * @param delta the delta by which to modify the counter (+ or -)
+      * @param immediate if true, the counter will be updated immediately,
+      * without regard to whether the ProcessSession is commit or rolled back;
+      * otherwise, the counter will be incremented only if and when the
+      * ProcessSession is committed.
+      */
+     void adjustCounter(String name, long delta, boolean immediate);
+ 
+     /**
+      * @return FlowFile that is next highest priority FlowFile to process.
+      * Otherwise returns null.
+      */
+     FlowFile get();
+ 
+     /**
+      * Returns up to <code>maxResults</code> FlowFiles from the work queue. If
+      * no FlowFiles are available, returns an empty list. Will not return null.
+      * If multiple incoming queues are present, the behavior is unspecified in
+      * terms of whether all queues or only a single queue will be polled in a
+      * single call.
+      *
+      * @param maxResults the maximum number of FlowFiles to return
+      * @return
+      * @throws IllegalArgumentException if <code>maxResults</code> is less than
+      * 0
+      */
+     List<FlowFile> get(int maxResults);
+ 
+     /**
+      * <p>
+      * Returns all FlowFiles from all of the incoming queues for which the given
+      * {@link FlowFileFilter} indicates should be accepted. Calls to this method
+      * provide exclusive access to the underlying queues. I.e., no other thread
+      * will be permitted to pull FlowFiles from this Processor's queues or add
+      * FlowFiles to this Processor's incoming queues until this method call has
+      * returned.
+      * </p>
+      *
+      * @param filter
+      * @return
+      */
+     List<FlowFile> get(FlowFileFilter filter);
+ 
+     /**
+      * @return the QueueSize that represents the number of FlowFiles and their
+      * combined data size for all FlowFiles waiting to be processed by the
+      * Processor that owns this ProcessSession, regardless of which Connection
+      * the FlowFiles live on
+      */
+     QueueSize getQueueSize();
+ 
+     /**
 -     * @return the set of all relationships for which space is available to
 -     * receive new objects
 -     */
 -    Set<Relationship> getAvailableRelationships();
 -
 -    /**
+      * Creates a new FlowFile in the repository with no content and without any
+      * linkage to a parent FlowFile. This method is appropriate only when data
+      * is received or created from an external system. Otherwise, this method
+      * should be avoided and should instead use {@link #create(FlowFile)} or
+      * {@link #create(Collection<FlowFile>)}.
+      *
+      * When this method is used, a Provenance CREATE or RECEIVE Event should be
+      * generated. See the {@link #getProvenanceReporter()} method and
+      * {@link ProvenanceReporter} class for more information
+      *
+      * @return newly created FlowFile
+      */
+     FlowFile create();
+ 
+     /**
+      * Creates a new FlowFile in the repository with no content but with a
+      * parent linkage to <code>parent</code>. The newly created FlowFile will
+      * inherit all of the parent's attributes except for the UUID. This method
+      * will automatically generate a Provenance FORK event or a Provenance JOIN
+      * event, depending on whether or not other FlowFiles are generated from the
+      * same parent before the ProcessSession is committed.
+      *
+      * @param parent
+      * @return
+      */
+     FlowFile create(FlowFile parent);
+ 
+     /**
+      * Creates a new FlowFile in the repository with no content but with a
+      * parent linkage to the FlowFiles specified by the parents Collection. The
+      * newly created FlowFile will inherit all of the attributes that are in
+      * common to all parents (except for the UUID, which will be in common if
+      * only a single parent exists). This method will automatically generate a
+      * Provenance JOIN event.
+      *
+      * @param parents
+      * @return
+      */
+     FlowFile create(Collection<FlowFile> parents);
+ 
+     /**
+      * Creates a new FlowFile that is a clone of the given FlowFile as of the
+      * time this is called, both in content and attributes. This method
+      * automatically emits a Provenance CLONE Event.
+      *
+      * @param example FlowFile to be the source of cloning - given FlowFile must
+      * be a part of the given session
+      * @return FlowFile that is a clone of the given example
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      * @throws NullPointerException if the argument null
+      */
+     FlowFile clone(FlowFile example);
+ 
+     /**
+      * Creates a new FlowFile whose parent is the given FlowFile. The content of
+      * the new FlowFile will be a subset of the byte sequence of the given
+      * FlowFile starting at the specified offset and with the length specified.
+      * The new FlowFile will contain all of the attributes of the original. This
+      * method automatically emits a Provenance FORK Event (or a Provenance CLONE
+      * Event, if the offset is 0 and the size is exactly equal to the size of
+      * the example FlowFile).
+      *
+      * @param example
+      * @param offset
+      * @param size
+      * @return a FlowFile with the specified size whose parent is first argument
+      * to this function
+      *
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session, or if the
+      * specified offset + size exceeds that of the size of the example FlowFile.
+      * Automatic rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      */
+     FlowFile clone(FlowFile example, long offset, long size);
+ 
+     /**
+      * Sets a penalty for the given FlowFile which will make it unavailable to
+      * be operated on any further during the penalty period.
+      *
+      * @param flowFile to penalize
+      * @return FlowFile the new FlowFile reference to use
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if the argument null
+      */
+     FlowFile penalize(FlowFile flowFile);
+ 
+     /**
+      * Updates the given FlowFiles attributes with the given key/value pair. If
+      * the key is named {@code uuid}, this attribute will be ignored.
+      *
+      * @param flowFile to update
+      * @param key of attribute
+      * @param value of attribute
+      * @return FlowFile the updated FlowFile
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if an argument is null
+      */
+     FlowFile putAttribute(FlowFile flowFile, String key, String value);
+ 
+     /**
+      * Updates the given FlowFiles attributes with the given key/value pairs. If
+      * the map contains a key named {@code uuid}, this attribute will be
+      * ignored.
+      *
+      * @param flowFile to update
+      * @param attributes the attributes to add to the given FlowFile
+      * @return FlowFile the updated FlowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if an argument is null
+      */
+     FlowFile putAllAttributes(FlowFile flowFile, Map<String, String> attributes);
+ 
+     /**
+      * Removes the given FlowFile attribute with the given key. If the key is
+      * named {@code uuid}, this method will return the same FlowFile without
+      * removing any attribute.
+      *
+      * @param flowFile to update
+      * @param key of attribute
+      * @return FlowFile the updated FlowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if the argument null
+      */
+     FlowFile removeAttribute(FlowFile flowFile, String key);
+ 
+     /**
+      * Removes the attributes with the given keys from the given FlowFile. If
+      * the set of keys contains the value {@code uuid}, this key will be ignored
+      *
+      * @param flowFile to update
+      * @param keys of attribute
+      * @return FlowFile the updated FlowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if the argument null
+      */
+     FlowFile removeAllAttributes(FlowFile flowFile, Set<String> keys);
+ 
+     /**
+      * Remove all attributes from the given FlowFile that have keys which match
+      * the given pattern. If the pattern matches the key {@code uuid}, this key
+      * will not be removed.
+      *
+      * @param flowFile to update
+      * @param keyPattern may be null; if supplied is matched against each of the
+      * FlowFile attribute keys
+      * @return FlowFile containing only attributes which did not meet the key
+      * pattern
+      */
+     FlowFile removeAllAttributes(FlowFile flowFile, Pattern keyPattern);
+ 
+     /**
+      * Transfers the given FlowFile to the appropriate destination processor
+      * work queue(s) based on the given relationship. If the relationship leads
+      * to more than one destination the state of the FlowFile is replicated such
+      * that each destination receives an exact copy of the FlowFile though each
+      * will have its own unique identity. The destination processors will not be
+      * able to operate on the given FlowFile until this session is committed or
+      * until the ownership of the session is migrated to another processor. If
+      * ownership of the session is passed to a destination processor then that
+      * destination processor will have immediate visibility of the transferred
+      * FlowFiles within the session.
+      *
+      * @param flowFile
+      * @param relationship
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if the argument null
+      * @throws IllegalArgumentException if given relationship is not a known or
+      * registered relationship
+      */
+     void transfer(FlowFile flowFile, Relationship relationship);
+ 
+     /**
+      * Transfers the given FlowFile back to the work queue from which it was
+      * pulled. The processor will not be able to operate on the given FlowFile
+      * until this session is committed. Any modifications that have been made to
+      * the FlowFile will be maintained. FlowFiles that are created by the
+      * processor cannot be transferred back to themselves via this method.
+      *
+      * @param flowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws IllegalArgumentException if the FlowFile was created by this
+      * processor
+      * @throws NullPointerException if the argument null
+      */
+     void transfer(FlowFile flowFile);
+ 
+     /**
+      * Transfers the given FlowFiles back to the work queues from which the
+      * FlowFiles were pulled. The processor will not be able to operate on the
+      * given FlowFile until this session is committed. Any modifications that
+      * have been made to the FlowFile will be maintained. FlowFiles that are
+      * created by the processor cannot be transferred back to themselves via
+      * this method.
+      *
+      * @param flowFiles
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFiles are already
+      * transferred or removed or don't belong to this session. Automatic
+      * rollback will occur.
+      * @throws IllegalArgumentException if the FlowFile was created by this
+      * processor
+      * @throws NullPointerException if the argument null
+      */
+     void transfer(Collection<FlowFile> flowFiles);
+ 
+     /**
+      * Transfers the given FlowFile to the appropriate destination processor
+      * work queue(s) based on the given relationship. If the relationship leads
+      * to more than one destination the state of the FlowFile is replicated such
+      * that each destination receives an exact copy of the FlowFile though each
+      * will have its own unique identity. The destination processors will not be
+      * able to operate on the given FlowFile until this session is committed or
+      * until the ownership of the session is migrated to another processor. If
+      * ownership of the session is passed to a destination processor then that
+      * destination processor will have immediate visibility of the transferred
+      * FlowFiles within the session.
+      *
+      * @param flowFiles
+      * @param relationship
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws NullPointerException if the argument null
+      * @throws IllegalArgumentException if given relationship is not a known or
+      * registered relationship
+      */
+     void transfer(Collection<FlowFile> flowFiles, Relationship relationship);
+ 
+     /**
+      * Ends the managed persistence for the given FlowFile. The persistent
+      * attributes for the FlowFile are deleted and so is the content assuming
+      * nothing else references it and this FlowFile will no longer be available
+      * for further operation.
+      *
+      * @param flowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      */
+     void remove(FlowFile flowFile);
+ 
+     /**
+      * Ends the managed persistence for the given FlowFiles. The persistent
+      * attributes for the FlowFile are deleted and so is the content assuming
+      * nothing else references it and this FlowFile will no longer be available
+      * for further operation.
+      *
+      * @param flowFiles
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if any of the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      */
+     void remove(Collection<FlowFile> flowFiles);
+ 
+     /**
+      * Executes the given callback against the contents corresponding to the
+      * given FlowFile.
+      *
+      * @param source
+      * @param reader
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     void read(FlowFile source, InputStreamCallback reader);
+ 
+     /**
+      * Combines the content of all given source FlowFiles into a single given
+      * destination FlowFile.
+      *
+      * @param sources
+      * @param destination
+      * @return updated destination FlowFile (new size, etc...)
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws IllegalArgumentException if the given destination is contained
+      * within the sources
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content. The state of the destination will be as it was prior to
+      * this call.
+      */
+     FlowFile merge(Collection<FlowFile> sources, FlowFile destination);
+ 
+     /**
+      * Combines the content of all given source FlowFiles into a single given
+      * destination FlowFile.
+      *
+      * @param sources
+      * @param destination
+      * @param header bytes that will be added to the beginning of the merged
+      * output. May be null or empty.
+      * @param footer bytes that will be added to the end of the merged output.
+      * May be null or empty.
+      * @param demarcator bytes that will be placed in between each object merged
+      * together. May be null or empty.
+      * @return updated destination FlowFile (new size, etc...)
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws IllegalArgumentException if the given destination is contained
+      * within the sources
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content. The state of the destination will be as it was prior to
+      * this call.
+      */
+     FlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator);
+ 
+     /**
+      * Executes the given callback against the content corresponding to the
+      * given FlowFile
+      *
+      * @param source
+      * @param writer
+      * @return updated FlowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be referenced, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     FlowFile write(FlowFile source, OutputStreamCallback writer);
+ 
+     /**
+      * Executes the given callback against the content corresponding to the
+      * given flow file
+      *
+      * @param source
+      * @param writer
+      * @return updated FlowFile
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     FlowFile write(FlowFile source, StreamCallback writer);
+ 
+     /**
+      * Executes the given callback against the content corresponding to the
+      * given FlowFile, such that any data written to the OutputStream of the
+      * content will be appended to the end of FlowFile.
+      *
+      * @param source
+      * @param writer
+      * @return
+      */
+     FlowFile append(FlowFile source, OutputStreamCallback writer);
+ 
+     /**
+      * Writes to the given FlowFile all content from the given content path.
+      *
+      * @param source the file from which content will be obtained
+      * @param keepSourceFile if true the content is simply copied; if false the
+      * original content might be used in a destructive way for efficiency such
+      * that the repository will have the data but the original data will be
+      * gone. If false the source object will be removed or gone once imported.
+      * It will not be restored if the session is rolled back so this must be
+      * used with caution. In some cases it can result in tremendous efficiency
+      * gains but is also dangerous.
+      * @param destination the FlowFile whose content will be updated
+      * @return the updated destination FlowFile (new size)
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     FlowFile importFrom(Path source, boolean keepSourceFile, FlowFile destination);
+ 
+     /**
+      * Writes to the given FlowFile all content from the given content path.
+      *
+      * @param source the file from which content will be obtained
+      * @param destination the FlowFile whose content will be updated
+      * @return the updated destination FlowFile (new size)
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     FlowFile importFrom(InputStream source, FlowFile destination);
+ 
+     /**
+      * Writes the content of the given FlowFile to the given destination path.
+      *
+      * @param flowFile
+      * @param destination
+      * @param append if true will append to the current content at the given
+      * path; if false will replace any current content
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     void exportTo(FlowFile flowFile, Path destination, boolean append);
+ 
+     /**
+      * Writes the content of the given FlowFile to the given destination stream
+      *
+      * @param flowFile
+      * @param destination
+      * @throws IllegalStateException if detected that this method is being
+      * called from within a callback of another method in this session and for
+      * the given FlowFile(s)
+      * @throws FlowFileHandlingException if the given FlowFile is already
+      * transferred or removed or doesn't belong to this session. Automatic
+      * rollback will occur.
+      * @throws MissingFlowFileException if the given FlowFile content cannot be
+      * found. The FlowFile should no longer be reference, will be internally
+      * destroyed, and the session is automatically rolled back and what is left
+      * of the FlowFile is destroyed.
+      * @throws FlowFileAccessException if some IO problem occurs accessing
+      * FlowFile content
+      */
+     void exportTo(FlowFile flowFile, OutputStream destination);
+ 
+     /**
+      * Returns a ProvenanceReporter that is tied to this ProcessSession.
+      *
+      * @return
+      */
+     ProvenanceReporter getProvenanceReporter();
+ }