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/04/28 16:53:33 UTC

[1/7] incubator-nifi git commit: NIFI-533: Initial implementation of FetchHDFS and ListHDFS

Repository: incubator-nifi
Updated Branches:
  refs/heads/ListHDFS [created] 00b686b0b


NIFI-533: Initial implementation of FetchHDFS and ListHDFS


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

Branch: refs/heads/ListHDFS
Commit: 94945a6fd5e88c083f1f3e32d3b3d5d5954686d1
Parents: 6fa5968
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Apr 24 20:13:21 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Apr 24 20:13:21 2015 -0400

----------------------------------------------------------------------
 .../notification/OnPrimaryNodeStateChange.java  |  44 ++
 .../notification/PrimaryNodeState.java          |  33 ++
 .../apache/nifi/controller/FlowController.java  |  16 +-
 .../org/apache/nifi/web/util/SnippetUtils.java  |   2 +-
 .../nifi-hdfs-processors/pom.xml                |   8 +-
 .../hadoop/AbstractHadoopProcessor.java         |  27 ++
 .../nifi/processors/hadoop/FetchHDFS.java       | 126 +++++
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  36 +-
 .../apache/nifi/processors/hadoop/ListHDFS.java | 466 +++++++++++++++++++
 .../processors/hadoop/util/HDFSListing.java     |  83 ++++
 .../nifi/processors/hadoop/util/LongSerDe.java  |  48 ++
 .../processors/hadoop/util/StringSerDe.java     |  44 ++
 .../org.apache.nifi.processor.Processor         |   6 +-
 .../standard/TestDetectDuplicate.java           |   5 +
 .../cache/client/DistributedMapCacheClient.java |  14 +
 .../DistributedMapCacheClientService.java       |  22 +
 .../distributed/cache/server/map/MapCache.java  |   1 +
 .../cache/server/map/MapCacheServer.java        |   7 +
 .../cache/server/map/PersistentMapCache.java    |  24 +
 .../cache/server/map/SimpleMapCache.java        |  22 +
 20 files changed, 996 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/OnPrimaryNodeStateChange.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/OnPrimaryNodeStateChange.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/OnPrimaryNodeStateChange.java
new file mode 100644
index 0000000..e073660
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/OnPrimaryNodeStateChange.java
@@ -0,0 +1,44 @@
+/*
+ * 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.annotation.notification;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * <p>
+ * Marker annotation that a component can use to indicate that a method should be 
+ * called whenever the state of the Primary Node in a cluster has changed.
+ * </p>
+ * 
+ * <p>
+ * Methods with this annotation should take either no arguments or one argument of type
+ * {@link PrimaryNodeState}. The {@link PrimaryNodeState} provides context about what changed
+ * so that the component can take appropriate action.
+ * </p>
+ */
+@Documented
+@Target({ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@Inherited
+public @interface OnPrimaryNodeStateChange {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/PrimaryNodeState.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/PrimaryNodeState.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/PrimaryNodeState.java
new file mode 100644
index 0000000..3a7245c
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/notification/PrimaryNodeState.java
@@ -0,0 +1,33 @@
+/*
+ * 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.annotation.notification;
+
+/**
+ * Represents a state change that occurred for the Primary Node of a NiFi cluster.
+ */
+public enum PrimaryNodeState {
+	/**
+	 * The node receiving this state has been elected the Primary Node of the NiFi cluster.
+	 */
+	ELECTED_PRIMARY_NODE,
+	
+	/**
+	 * The node receiving this state was the Primary Node but has now had its Primary Node
+	 * role revoked.
+	 */
+	PRIMARY_NODE_REVOKED;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index ec25ab1..ef9fe77 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -53,6 +53,8 @@ import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange;
+import org.apache.nifi.annotation.notification.PrimaryNodeState;
 import org.apache.nifi.cluster.BulletinsPayload;
 import org.apache.nifi.cluster.HeartbeatPayload;
 import org.apache.nifi.cluster.protocol.DataFlow;
@@ -74,8 +76,8 @@ import org.apache.nifi.connectable.Position;
 import org.apache.nifi.connectable.Size;
 import org.apache.nifi.connectable.StandardConnection;
 import org.apache.nifi.controller.exception.CommunicationsException;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.exception.ComponentLifeCycleException;
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.label.StandardLabel;
 import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
@@ -3098,6 +3100,18 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
             LOG.info("Setting primary flag from '" + this.primary + "' to '" + primary + "'");
 
+            final PrimaryNodeState nodeState = primary ? PrimaryNodeState.ELECTED_PRIMARY_NODE : PrimaryNodeState.PRIMARY_NODE_REVOKED;
+            final ProcessGroup rootGroup = getGroup(getRootGroupId());
+            for (final ProcessorNode procNode : rootGroup.findAllProcessors()) {
+            	ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, procNode.getProcessor(), nodeState);
+            }
+            for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
+            	ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, serviceNode.getControllerServiceImplementation(), nodeState);
+            }
+            for (final ReportingTaskNode reportingTaskNode : getAllReportingTasks()) {
+            	ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, reportingTaskNode.getReportingTask(), nodeState);
+            }
+            
             // update primary
             this.primary = primary;
             eventDrivenWorkerQueue.setPrimary(primary);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
index 76789c6..d8cb69a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
@@ -257,7 +257,7 @@ public final class SnippetUtils {
             final PropertyDescriptor descriptor = entry.getKey();
             final String propertyValue = entry.getValue();
             
-            if ( descriptor.getControllerServiceDefinition() != null ) {
+            if ( descriptor.getControllerServiceDefinition() != null && propertyValue != null ) {
                 final ControllerServiceNode referencedNode = flowController.getControllerServiceNode(propertyValue);
                 if ( referencedNode == null ) {
                     throw new IllegalStateException("Controller Service with ID " + propertyValue + " is referenced in template but cannot be found");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
index 3ff1e88..ede32ab 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
@@ -42,8 +42,12 @@
             <groupId>org.apache.hadoop</groupId> 
             <artifactId>hadoop-common</artifactId> 
             <scope>provided</scope> 
-        </dependency> 
-        <dependency> 
+        </dependency>
+        <dependency>
+        	<groupId>org.apache.nifi</groupId>
+        	<artifactId>nifi-distributed-cache-client-service-api</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hadoop</groupId> 
             <artifactId>hadoop-hdfs</artifactId> 
             <scope>provided</scope> 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 8d5749b..44ebbf8 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -217,4 +217,31 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
         };
     }
 
+    
+    /**
+     * Returns the relative path of the child that does not include the filename
+     * or the root path.
+     * @param root
+     * @param child
+     * @return 
+     */
+    public static String getPathDifference(final Path root, final Path child) {
+        final int depthDiff = child.depth() - root.depth();
+        if (depthDiff <= 1) {
+            return "".intern();
+        }
+        String lastRoot = root.getName();
+        Path childsParent = child.getParent();
+        final StringBuilder builder = new StringBuilder();
+        builder.append(childsParent.getName());
+        for (int i = (depthDiff - 3); i >= 0; i--) {
+            childsParent = childsParent.getParent();
+            String name = childsParent.getName();
+            if (name.equals(lastRoot) && childsParent.toString().endsWith(root.toString())) {
+                break;
+            }
+            builder.insert(0, Path.SEPARATOR).insert(0, name);
+        }
+        return builder.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
new file mode 100644
index 0000000..06bb3c6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -0,0 +1,126 @@
+/*
+ * 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.hadoop;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.util.StopWatch;
+
+@SupportsBatching
+@Tags({"hadoop", "hdfs", "get", "ingest", "fetch", "source"})
+@CapabilityDescription("Retrieves a file from HDFS. The content of the incoming FlowFile is replaced by the content of the file in HDFS. "
+		+ "The file in HDFS is left intact without any changes being made to it.")
+@WritesAttribute(attribute="hdfs.failure.reason", description="When a FlowFile is routed to 'failure', this attribute is added indicating why the file could "
+		+ "not be fetched from HDFS")
+@SeeAlso({ListHDFS.class, GetHDFS.class, PutHDFS.class})
+public class FetchHDFS extends AbstractHadoopProcessor {
+
+	static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()
+		.name("HDFS Filename")
+		.description("The name of the HDFS file to retrieve")
+		.required(true)
+		.expressionLanguageSupported(true)
+		.defaultValue("${path}/${filename}")
+		.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+		.build();
+	
+	static final Relationship REL_SUCCESS = new Relationship.Builder()
+		.name("success")
+		.description("FlowFiles will be routed to this relationship once they have been updated with the content of the HDFS file")
+		.build();
+	static final Relationship REL_FAILURE = new Relationship.Builder()
+		.name("failure")
+		.description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieved and trying again will likely not be helpful. "
+				+ "This would occur, for instance, if the file is not found or if there is a permissions issue")
+		.build();
+	static final Relationship REL_COMMS_FAILURE = new Relationship.Builder()
+		.name("comms.failure")
+		.description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieve due to a communications failure. "
+				+ "This generally indicates that the Fetch should be tried again.")
+		.build();
+
+	@Override
+	protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+		final List<PropertyDescriptor> properties = new ArrayList<>();
+		properties.add(HADOOP_CONFIGURATION_RESOURCES);
+		properties.add(FILENAME);
+		return properties;
+	}
+	
+	@Override
+	public Set<Relationship> getRelationships() {
+		final Set<Relationship> relationships = new HashSet<>();
+		relationships.add(REL_SUCCESS);
+		relationships.add(REL_FAILURE);
+		relationships.add(REL_COMMS_FAILURE);
+		return relationships;
+	}
+	
+	@Override
+	public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+		FlowFile flowFile = session.get();
+		if ( flowFile == null ) {
+			return;
+		}
+		
+		final FileSystem hdfs = hdfsResources.get().getValue();
+		final Path path = new Path(context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue());
+		final URI uri = path.toUri();
+		
+		final StopWatch stopWatch = new StopWatch(true);
+		try (final FSDataInputStream inStream = hdfs.open(path, 16384)) {
+			flowFile = session.importFrom(inStream, flowFile);
+			stopWatch.stop();
+			getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
+			session.getProvenanceReporter().modifyContent(flowFile, "Fetched content from " + uri, stopWatch.getDuration(TimeUnit.MILLISECONDS));
+			session.transfer(flowFile, REL_SUCCESS);
+		} catch (final FileNotFoundException | AccessControlException e) {
+			getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
+			flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
+			flowFile = session.penalize(flowFile);
+			session.transfer(flowFile, REL_FAILURE);
+		} catch (final IOException e) {
+			getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
+			flowFile = session.penalize(flowFile);
+			session.transfer(flowFile, REL_COMMS_FAILURE);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
index d763c29..1dd5b91 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
@@ -58,16 +58,13 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.util.StopWatch;
 
-/**
- * This processor reads files from HDFS into NiFi FlowFiles.
- */
 @TriggerWhenEmpty
 @Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"})
-@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles")
+@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles. This Processor will delete the file from HDFS after fetching it.")
 @WritesAttributes({
         @WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."),
         @WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".") })
-@SeeAlso(PutHDFS.class)
+@SeeAlso({PutHDFS.class, ListHDFS.class})
 public class GetHDFS extends AbstractHadoopProcessor {
 
     public static final String BUFFER_SIZE_KEY = "io.file.buffer.size";
@@ -104,7 +101,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
 
     public static final PropertyDescriptor KEEP_SOURCE_FILE = new PropertyDescriptor.Builder()
             .name("Keep Source File")
-            .description("Determines whether to delete the file from HDFS after it has been successfully transferred")
+            .description("Determines whether to delete the file from HDFS after it has been successfully transferred. If true, the file will be fetched repeatedly. This is intended for testing only.")
             .required(true)
             .allowableValues("true", "false")
             .defaultValue("false")
@@ -464,32 +461,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
         return files;
     }
 
-    /**
-     * Returns the relative path of the child that does not include the filename
-     * or the root path.
-     * @param root
-     * @param child
-     * @return 
-     */
-    public static String getPathDifference(final Path root, final Path child) {
-        final int depthDiff = child.depth() - root.depth();
-        if (depthDiff <= 1) {
-            return "".intern();
-        }
-        String lastRoot = root.getName();
-        Path childsParent = child.getParent();
-        final StringBuilder builder = new StringBuilder();
-        builder.append(childsParent.getName());
-        for (int i = (depthDiff - 3); i >= 0; i--) {
-            childsParent = childsParent.getParent();
-            String name = childsParent.getName();
-            if (name.equals(lastRoot) && childsParent.toString().endsWith(root.toString())) {
-                break;
-            }
-            builder.insert(0, Path.SEPARATOR).insert(0, name);
-        }
-        return builder.toString();
-    }
+    
 
     /**
      * Holder for a snapshot in time of some processor properties that are

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
new file mode 100644
index 0000000..707b50d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
@@ -0,0 +1,466 @@
+/*
+ * 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.hadoop;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange;
+import org.apache.nifi.annotation.notification.PrimaryNodeState;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+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.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.hadoop.util.HDFSListing;
+import org.apache.nifi.processors.hadoop.util.StringSerDe;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+
+
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"hadoop", "HDFS", "get", "list", "ingest", "source", "filesystem"})
+@CapabilityDescription("Retrieves a listing of files from HDFS. For each file that is listed in HDFS, creates a FlowFile that represents "
+		+ "the HDFS file so that it can be fetched in conjunction with ListHDFS. This Processor is designed to run on Primary Node only "
+		+ "in a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left off without duplicating "
+		+ "all of the data. Unlike GetHDFS, this Processor does not delete any data from HDFS.")
+@WritesAttributes({
+        @WritesAttribute(attribute="filename", description="The name of the file that was read from HDFS."),
+        @WritesAttribute(attribute="path", description="The path is set to the absolute path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"/tmp/abc/1/2/3\"."),
+		@WritesAttribute(attribute="hdfs.owner", description="The user that owns the file in HDFS"),
+		@WritesAttribute(attribute="hdfs.group", description="The group that owns the file in HDFS"),
+		@WritesAttribute(attribute="hdfs.lastModified", description="The timestamp of when the file in HDFS was last modified, as milliseconds since midnight Jan 1, 1970 UTC"),
+		@WritesAttribute(attribute="hdfs.length", description="The number of bytes in the file in HDFS"),
+		@WritesAttribute(attribute="hdfs.replication", description="The number of HDFS replicas for hte file"),
+		@WritesAttribute(attribute="hdfs.permissions", description="The permissions for the file in HDFS. This is formatted as 3 characters for the owner, 3 for the group, and 3 for other users. For example rw-rw-r--")
+})
+@SeeAlso({GetHDFS.class, FetchHDFS.class, PutHDFS.class})
+public class ListHDFS extends AbstractHadoopProcessor {
+
+	public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
+		.name("Distributed Cache Service")
+		.description("Specifies the Controller Service that should be used to maintain state about what has been pulled from HDFS so that if a new node begins pulling data, it won't duplicate all of the work that has been done.")
+		.required(true)
+		.identifiesControllerService(DistributedMapCacheClient.class)
+		.build();
+
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+	    .name(DIRECTORY_PROP_NAME)
+	    .description("The HDFS directory from which files should be read")
+	    .required(true)
+	    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+	    .build();
+	
+	public static final PropertyDescriptor RECURSE_SUBDIRS = new PropertyDescriptor.Builder()
+	    .name("Recurse Subdirectories")
+	    .description("Indicates whether to list files from subdirectories of the HDFS directory")
+	    .required(true)
+	    .allowableValues("true", "false")
+	    .defaultValue("true")
+	    .build();
+	
+	
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+	    .name("success")
+	    .description("All FlowFiles are transferred to this relationship")
+	    .build();
+
+    private volatile Long lastListingTime = null;
+    private volatile Set<Path> latestPathsListed = new HashSet<>();
+    private volatile boolean electedPrimaryNode = false;
+    private File persistenceFile = null;
+    
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+    	super.init(context);
+    	persistenceFile = new File("conf/state/" + getIdentifier());
+    }
+	
+	@Override
+	protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+		final List<PropertyDescriptor> properties = new ArrayList<>();
+		properties.add(HADOOP_CONFIGURATION_RESOURCES);
+		properties.add(DISTRIBUTED_CACHE_SERVICE);
+		properties.add(DIRECTORY);
+		properties.add(RECURSE_SUBDIRS);
+		return properties;
+	}
+	
+	@Override
+	public Set<Relationship> getRelationships() {
+		final Set<Relationship> relationships = new HashSet<>();
+		relationships.add(REL_SUCCESS);
+		return relationships;
+	}
+
+	private String getKey(final String directory) {
+		return getIdentifier() + ".lastListingTime." + directory;
+	}
+	
+	@OnPrimaryNodeStateChange
+	public void onPrimaryNodeChange(final PrimaryNodeState newState) {
+		if ( newState == PrimaryNodeState.ELECTED_PRIMARY_NODE ) {
+			electedPrimaryNode = true;
+		}
+	}
+	
+	@Override
+	public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+		if ( descriptor.equals(DIRECTORY) ) {
+			lastListingTime = null;	// clear lastListingTime so that we have to fetch new time
+			latestPathsListed = new HashSet<>();
+		}
+	}
+	
+	private HDFSListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException {
+		final ObjectMapper mapper = new ObjectMapper();
+        final JsonNode jsonNode = mapper.readTree(serializedState);
+        return mapper.readValue(jsonNode, HDFSListing.class);
+	}
+	
+	
+	@Override
+	public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+		final String directory = context.getProperty(DIRECTORY).getValue();
+
+		// Determine the timestamp for the last file that we've listed.
+		Long minTimestamp = lastListingTime;
+		if ( minTimestamp == null || electedPrimaryNode ) {
+			// We haven't yet restored any state from local or distributed state - or it's been at least a minute since
+			// we have performed a listing. In this case, 
+			// First, attempt to get timestamp from distributed cache service.
+			final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
+			
+			try {
+				final StringSerDe serde = new StringSerDe();
+				final String serializedState = client.get(getKey(directory), serde, serde);
+				if ( serializedState == null || serializedState.isEmpty() ) {
+					minTimestamp = null;
+					this.latestPathsListed = Collections.emptySet();
+				} else {
+					final HDFSListing listing = deserialize(serializedState);
+					this.lastListingTime = listing.getLatestTimestamp().getTime();
+					minTimestamp = listing.getLatestTimestamp().getTime();
+					this.latestPathsListed = listing.toPaths();
+				}
+				
+				this.lastListingTime = minTimestamp;
+				electedPrimaryNode = false;	// no requirement to pull an update from the distributed cache anymore.
+			} catch (final IOException ioe) {
+				getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished.");
+				context.yield();
+				return;
+			}
+			
+			// Check the persistence file. We want to use the latest timestamp that we have so that
+			// we don't duplicate data.
+			try {
+				if ( persistenceFile.exists() ) {
+					try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
+						final Properties props = new Properties();
+						props.load(fis);
+
+						// get the local timestamp for this directory, if it exists.
+						final String locallyPersistedValue = props.getProperty(directory);
+						if ( locallyPersistedValue != null ) {
+							final HDFSListing listing = deserialize(locallyPersistedValue);
+							final long localTimestamp = listing.getLatestTimestamp().getTime();
+							
+							// If distributed state doesn't have an entry or the local entry is later than the distributed state,
+							// update the distributed state so that we are in sync.
+							if (minTimestamp == null || localTimestamp > minTimestamp) {
+								minTimestamp = localTimestamp;
+								
+								// Our local persistence file shows a later time than the Distributed service.
+								// Update the distributed service to match our local state.
+								try {
+									final StringSerDe serde = new StringSerDe();
+									client.put(getKey(directory), locallyPersistedValue, serde, serde);
+								} catch (final IOException ioe) {
+									getLogger().warn("Local timestamp for {} is {}, which is later than Distributed state but failed to update Distributed "
+											+ "state due to {}. If a new node performs HDFS Listing, data duplication may occur", 
+											new Object[] {directory, locallyPersistedValue, ioe});
+								}
+							}
+						}
+					}
+				}
+			} catch (final IOException ioe) {
+				getLogger().warn("Failed to recover local state due to {}. Assuming that the state from the distributed cache is correct.", ioe);
+			}
+		}
+		
+		
+		// Pull in any file that is newer than the timestamp that we have.
+		final FileSystem hdfs = hdfsResources.get().getValue();
+		final boolean recursive = context.getProperty(RECURSE_SUBDIRS).asBoolean();
+		final Path rootPath = new Path(directory);
+		
+		int listCount = 0;
+		Long latestListingModTime = null;
+		final Set<FileStatus> statuses;
+		try {
+			statuses = getStatuses(rootPath, recursive, hdfs);
+			for ( final FileStatus status : statuses ) {
+				// don't get anything where the last modified timestamp is equal to our current timestamp.
+				// if we do, then we run the risk of multiple files having the same last mod date but us only
+				// seeing a portion of them.
+				// I.e., there could be 5 files with last mod date = (now). But if we do the listing now, maybe
+				// only 2 exist and 3 more will exist later in this millisecond. So we ignore anything with a
+				// modified date not before the current time.
+				final long fileModTime = status.getModificationTime();
+				
+				// we only want the file if its timestamp is later than the minTimestamp or equal to and we didn't pull it last time.
+				// Also, HDFS creates files with the suffix _COPYING_ when they are being written - we want to ignore those.
+				boolean fetch = !status.getPath().getName().endsWith("_COPYING_") &&
+						(minTimestamp == null || fileModTime > minTimestamp || (fileModTime == minTimestamp && !latestPathsListed.contains(status.getPath())));
+				
+				// Create the FlowFile for this path.
+				if ( fetch ) {
+					final Map<String, String> attributes = createAttributes(status);
+					FlowFile flowFile = session.create();
+					flowFile = session.putAllAttributes(flowFile, attributes);
+					session.transfer(flowFile, REL_SUCCESS);
+					listCount++;
+					
+					if ( latestListingModTime == null || fileModTime > latestListingModTime ) {
+						latestListingModTime = fileModTime;
+					}
+				}
+			}
+		} catch (final IOException ioe) {
+			getLogger().error("Failed to perform listing of HDFS due to {}", new Object[] {ioe});
+			return;
+		}
+		
+		if ( listCount > 0 ) {
+			getLogger().info("Successfully created listing with {} new files from HDFS", new Object[] {listCount});
+			session.commit();
+
+			// We have performed a listing and pushed the FlowFiles out.
+			// Now, we need to persist state about the Last Modified timestamp of the newest file
+			// that we pulled in. We do this in order to avoid pulling in the same file twice.
+			// However, we want to save the state both locally and remotely.
+			// We store the state remotely so that if a new Primary Node is chosen, it can pick up where the
+			// previously Primary Node left off.
+			// We also store the state locally so that if the node is restarted, and the node cannot contact
+			// the distributed state cache, the node can continue to run (if it is primary node).
+			String serializedState = null;
+			try {
+				serializedState = serializeState(latestListingModTime, statuses);
+			} catch (final Exception e) {
+				getLogger().error("Failed to serialize state due to {}", new Object[] {e});
+			}
+			
+			if ( serializedState != null ) {
+				// Save our state locally.
+				try {
+					persistLocalState(directory, serializedState);
+				} catch (final IOException ioe) {
+					getLogger().warn("Unable to save state locally. If the node is restarted now, data may be duplicated. Failure is due to {}", ioe);
+				}
+	
+				// Attempt to save state to remote server.
+				final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
+				try {
+					client.put(getKey(directory), serializedState, new StringSerDe(), new StringSerDe());
+				} catch (final IOException ioe) {
+					getLogger().warn("Unable to communicate with distributed cache server due to {}. Persisting state locally instead.", ioe);
+				}
+			}
+			
+			lastListingTime = latestListingModTime;
+		} else {
+			getLogger().debug("There is no data to list. Yielding.");
+			context.yield();
+			
+			// lastListingTime = 0 so that we don't continually poll the distributed cache / local file system
+			if ( lastListingTime == null ) {
+				lastListingTime = 0L;
+			}
+			
+			return;
+		}
+	}
+	
+	private Set<FileStatus> getStatuses(final Path path, final boolean recursive, final FileSystem hdfs) throws IOException {
+		final Set<FileStatus> statusSet = new HashSet<>();
+		
+		final FileStatus[] statuses = hdfs.listStatus(path);
+		
+		for ( final FileStatus status : statuses ) {
+			if ( status.isDirectory() ) {
+				if ( recursive ) {
+					try {
+						statusSet.addAll(getStatuses(status.getPath(), recursive, hdfs));
+					} catch (final IOException ioe) {
+						getLogger().error("Failed to retrieve HDFS listing for subdirectory {} due to {}; will continue listing others", new Object[] {status.getPath(), ioe});
+					}
+				}
+			} else {
+				statusSet.add(status);
+			}
+		}
+		
+		return statusSet;
+	}
+	
+	
+	private String serializeState(final long latestListingTime, final Set<FileStatus> statuses) throws JsonGenerationException, JsonMappingException, IOException {
+		// we need to keep track of all files that we pulled in that had a modification time equal to
+		// lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
+		// that have a mod time equal to that timestamp because more files may come in with the same timestamp
+		// later in the same millisecond.
+		if ( statuses.isEmpty() ) {
+			return null;
+		} else {
+			final List<FileStatus> sortedStatuses = new ArrayList<>(statuses);
+			Collections.sort(sortedStatuses, new Comparator<FileStatus>() {
+				@Override
+				public int compare(final FileStatus o1, final FileStatus o2) {
+					return Long.compare(o1.getModificationTime(), o2.getModificationTime());
+				}
+			});
+			
+			final long latestListingModTime = sortedStatuses.get(sortedStatuses.size() - 1).getModificationTime();
+			final Set<Path> pathsWithModTimeEqualToListingModTime = new HashSet<>();
+			for (int i=sortedStatuses.size() - 1; i >= 0; i--) {
+				final FileStatus status = sortedStatuses.get(i);
+				if (status.getModificationTime() == latestListingModTime) {
+					pathsWithModTimeEqualToListingModTime.add(status.getPath());
+				}
+			}
+			
+			this.latestPathsListed = pathsWithModTimeEqualToListingModTime;
+			
+			final HDFSListing listing = new HDFSListing();
+			listing.setLatestTimestamp(new Date(latestListingModTime));
+			final Set<String> paths = new HashSet<>();
+			for ( final Path path : pathsWithModTimeEqualToListingModTime ) {
+				paths.add(path.toUri().toString());
+			}
+			listing.setMatchingPaths(paths);
+
+			final ObjectMapper mapper = new ObjectMapper();
+			final String serializedState = mapper.writerWithType(HDFSListing.class).writeValueAsString(listing);
+			return serializedState;
+		}
+	}
+	
+	private void persistLocalState(final String directory, final String serializedState) throws IOException {
+		// we need to keep track of all files that we pulled in that had a modification time equal to
+		// lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
+		// that have a mod time equal to that timestamp because more files may come in with the same timestamp
+		// later in the same millisecond.
+		final File dir = persistenceFile.getParentFile();
+		if ( !dir.exists() && !dir.mkdirs() ) {
+			throw new IOException("Could not create directory " + dir.getAbsolutePath() + " in order to save local state");
+		}
+		
+		final Properties props = new Properties();
+		if ( persistenceFile.exists() ) {
+			try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
+				props.load(fis);
+			}
+		}
+
+		props.setProperty(directory, serializedState);
+		
+		try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) {
+			props.store(fos, null);
+		}
+	}
+
+	private String getAbsolutePath(final Path path) {
+		final Path parent = path.getParent();
+		final String prefix = (parent == null || parent.getName().equals("")) ? "" : getAbsolutePath(parent);
+		return prefix + "/" + path.getName();
+	}
+	
+	private Map<String, String> createAttributes(final FileStatus status) {
+		final Map<String, String> attributes = new HashMap<>();
+		attributes.put(CoreAttributes.FILENAME.key(), status.getPath().getName());
+		attributes.put(CoreAttributes.PATH.key(), getAbsolutePath(status.getPath().getParent()));
+		
+		attributes.put("hdfs.owner", status.getOwner());
+		attributes.put("hdfs.group", status.getGroup());
+		attributes.put("hdfs.lastModified", String.valueOf(status.getModificationTime()));
+		attributes.put("hdfs.length", String.valueOf(status.getLen()));
+		attributes.put("hdfs.replication", String.valueOf(status.getReplication()));
+		
+		final FsPermission permission = status.getPermission();
+		final String perms = getPerms(permission.getUserAction()) + getPerms(permission.getGroupAction()) + getPerms(permission.getOtherAction());
+		attributes.put("hdfs.permissions", perms);
+		return attributes;
+	}
+	
+	private String getPerms(final FsAction action) {
+		final StringBuilder sb = new StringBuilder();
+		if ( action.implies(FsAction.READ) ) {
+			sb.append("r");
+		} else {
+			sb.append("-");
+		}
+		
+		if ( action.implies(FsAction.WRITE) ) {
+			sb.append("w");
+		} else {
+			sb.append("-");
+		}
+		
+		if ( action.implies(FsAction.EXECUTE) ) {
+			sb.append("x");
+		} else {
+			sb.append("-");
+		}
+		
+		return sb.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
new file mode 100644
index 0000000..9f4d68b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
@@ -0,0 +1,83 @@
+/*
+ * 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.hadoop.util;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlTransient;
+import javax.xml.bind.annotation.XmlType;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A simple POJO for maintaining state about the last HDFS Listing that was performed so that
+ * we can avoid pulling the same file multiple times 
+ */
+@XmlType(name = "listing")
+public class HDFSListing {
+	private Date latestTimestamp;
+	private Collection<String> matchingPaths;
+	
+	/**
+	 * @return the modification date of the newest file that was contained in the HDFS Listing
+	 */
+	public Date getLatestTimestamp() {
+		return latestTimestamp;
+	}
+	
+	/**
+	 * Sets the timestamp of the modification date of the newest file that was contained in the HDFS Listing
+	 * 
+	 * @param latestTimestamp the timestamp of the modification date of the newest file that was contained in the HDFS Listing
+	 */
+	public void setLatestTimestamp(Date latestTimestamp) {
+		this.latestTimestamp = latestTimestamp;
+	}
+	
+	/**
+	 * @return a Collection containing the paths of all files in the HDFS Listing whose Modification date
+	 * was equal to {@link #getLatestTimestamp()}
+	 */
+	@XmlTransient
+	public Collection<String> getMatchingPaths() {
+		return matchingPaths;
+	}
+	
+	/**
+	 * @return a Collection of {@link Path} objects equivalent to those returned by {@link #getMatchingPaths()}
+	 */
+	public Set<Path> toPaths() {
+		final Set<Path> paths = new HashSet<>(matchingPaths.size());
+		for ( final String pathname : matchingPaths ) {
+			paths.add(new Path(pathname));
+		}
+		return paths;
+	}
+
+	/**
+	 * Sets the Collection containing the paths of all files in the HDFS Listing whose Modification Date was
+	 * equal to {@link #getLatestTimestamp()}
+	 * @param matchingPaths
+	 */
+	public void setMatchingPaths(Collection<String> matchingPaths) {
+		this.matchingPaths = matchingPaths;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
new file mode 100644
index 0000000..ef0e590
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hadoop.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
+import org.apache.nifi.distributed.cache.client.exception.SerializationException;
+
+public class LongSerDe implements Serializer<Long>, Deserializer<Long> {
+
+	@Override
+	public Long deserialize(final byte[] input) throws DeserializationException, IOException {
+		if ( input == null || input.length == 0 ) {
+			return null;
+		}
+		
+		final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(input));
+		return dis.readLong();
+	}
+
+	@Override
+	public void serialize(final Long value, final OutputStream out) throws SerializationException, IOException {
+		final DataOutputStream dos = new DataOutputStream(out);
+		dos.writeLong(value);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
new file mode 100644
index 0000000..848831f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hadoop.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
+import org.apache.nifi.distributed.cache.client.exception.SerializationException;
+
+public class StringSerDe implements Serializer<String>, Deserializer<String> {
+
+	@Override
+	public String deserialize(final byte[] value) throws DeserializationException, IOException {
+		if ( value == null ) {
+			return null;
+		}
+		
+		return new String(value, StandardCharsets.UTF_8);
+	}
+
+	@Override
+	public void serialize(final String value, final OutputStream out) throws SerializationException, IOException {
+		out.write(value.getBytes(StandardCharsets.UTF_8));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index da16ef7..4b359e8 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -12,7 +12,9 @@
 # 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.
-org.apache.nifi.processors.hadoop.GetHDFS
-org.apache.nifi.processors.hadoop.PutHDFS
 org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile
+org.apache.nifi.processors.hadoop.FetchHDFS
+org.apache.nifi.processors.hadoop.GetHDFS
 org.apache.nifi.processors.hadoop.GetHDFSSequenceFile
+org.apache.nifi.processors.hadoop.ListHDFS
+org.apache.nifi.processors.hadoop.PutHDFS

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
index eed0d36..df7297a 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
@@ -178,6 +178,11 @@ public class TestDetectDuplicate {
             exists = false;
             return true;
         }
+
+		@Override
+		public <K, V> void put(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+			
+		}
     }
 
     private static class StringSerializer implements Serializer<String> {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
index 8283137..975dc63 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
@@ -83,6 +83,20 @@ public interface DistributedMapCacheClient extends ControllerService {
     <K> boolean containsKey(K key, Serializer<K> keySerializer) throws IOException;
 
     /**
+     * Adds the specified key and value to the cache, overwriting any value that is
+     * currently set.
+     * 
+     * @param key The key to set
+     * @param value The value to associate with the given Key
+     * @param keySerializer the Serializer that will be used to serialize the key into bytes
+     * @param valueSerializer the Serializer that will be used to serialize the value into bytes
+     * 
+     * @throws IOException if unable to communicate with the remote instance
+     * @throws NullPointerException if the key or either serializer is null
+     */
+    <K, V> void put(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException;
+    
+    /**
      * Returns the value in the cache for the given key, if one exists;
      * otherwise returns <code>null</code>
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
index 92bda8f..06ff42b 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
@@ -116,6 +116,28 @@ public class DistributedMapCacheClientService extends AbstractControllerService
         });
     }
 
+    public <K, V> void put(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+    	withCommsSession(new CommsAction<Object>() {
+			@Override
+			public Object execute(final CommsSession session) throws IOException {
+				final DataOutputStream dos = new DataOutputStream(session.getOutputStream());
+				dos.writeUTF("put");
+				
+				serialize(key, keySerializer, dos);
+				serialize(value, valueSerializer, dos);
+				
+				dos.flush();
+				final DataInputStream dis = new DataInputStream(session.getInputStream());
+				final boolean success = dis.readBoolean();
+				if ( !success ) {
+					throw new IOException("Expected to receive confirmation of 'put' request but received unexpected response");
+				}
+				
+				return null;
+			}
+    	});
+    }
+    
     @Override
     public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
         return withCommsSession(new CommsAction<Boolean>() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
index 534cb0b..8903046 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 public interface MapCache {
 
     MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException;
+    MapPutResult put(ByteBuffer key, ByteBuffer value) throws IOException;
     boolean containsKey(ByteBuffer key) throws IOException;
     ByteBuffer get(ByteBuffer key) throws IOException;
     ByteBuffer remove(ByteBuffer key) throws IOException;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
index e4a600e..cf8996c 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
@@ -65,6 +65,13 @@ public class MapCacheServer extends AbstractCacheServer {
                 dos.writeBoolean(putResult.isSuccessful());
                 break;
             }
+            case "put": {
+            	final byte[] key = readValue(dis);
+            	final byte[] value = readValue(dis);
+            	cache.put(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
+                dos.writeBoolean(true);
+            	break;
+            }
             case "containsKey": {
                 final byte[] key = readValue(dis);
                 final boolean contains = cache.containsKey(ByteBuffer.wrap(key));

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
index 77fb77d..82b1787 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
@@ -78,6 +78,30 @@ public class PersistentMapCache implements MapCache {
         
         return putResult;
     }
+    
+    @Override
+    public MapPutResult put(final ByteBuffer key, final ByteBuffer value) throws IOException {
+    	final MapPutResult putResult = wrapped.put(key, value);
+        if ( putResult.isSuccessful() ) {
+            // The put was successful.
+            final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value);
+            final List<MapWaliRecord> records = new ArrayList<>();
+            records.add(record);
+
+            if ( putResult.getEvictedKey() != null ) {
+                records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue()));
+            }
+            
+            wali.update(Collections.singletonList(record), false);
+            
+            final long modCount = modifications.getAndIncrement();
+            if ( modCount > 0 && modCount % 100000 == 0 ) {
+                wali.checkpoint();
+            }
+        }
+        
+        return putResult;
+    }
 
     @Override
     public boolean containsKey(final ByteBuffer key) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/94945a6f/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
index 10139f1..d8f9c45 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
@@ -106,6 +106,28 @@ public class SimpleMapCache implements MapCache {
         }
     }
     
+
+    @Override
+    public MapPutResult put(final ByteBuffer key, final ByteBuffer value) {
+        writeLock.lock();
+        try {
+        	// evict if we need to in order to make room for a new entry.
+            final MapCacheRecord evicted = evict();
+
+            final MapCacheRecord record = new MapCacheRecord(key, value);
+        	final MapCacheRecord existing = cache.put(key, record);
+        	inverseCacheMap.put(record, key);
+        	
+        	final ByteBuffer existingValue = (existing == null) ? null : existing.getValue();
+        	final ByteBuffer evictedKey = (evicted == null) ? null : evicted.getKey();
+        	final ByteBuffer evictedValue = (evicted == null) ? null : evicted.getValue();
+        	
+        	return new MapPutResult(true, key, value, existingValue, evictedKey, evictedValue);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+    
     @Override
     public boolean containsKey(final ByteBuffer key) {
         readLock.lock();


[6/7] incubator-nifi git commit: NIFI-395: Removed documentation dealing with how to configure controller-services.xml to use controller services, since they are now configured via the UI

Posted by ma...@apache.org.
NIFI-395: Removed documentation dealing with how to configure controller-services.xml to use controller services, since they are now configured via the UI


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

Branch: refs/heads/ListHDFS
Commit: 8c006237e308d3c8071cab1f1c529d640761ff6d
Parents: 62edd7a
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Apr 28 10:12:09 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Apr 28 10:12:09 2015 -0400

----------------------------------------------------------------------
 .../DistributedMapCacheClientService.java       |  2 +
 .../DistributedSetCacheClientService.java       |  2 +
 .../additionalDetails.html                      | 45 -------------
 .../additionalDetails.html                      | 46 --------------
 .../index.html                                  | 67 --------------------
 .../additionalDetails.html                      | 49 --------------
 6 files changed, 4 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
index 51138b9..7428e35 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
@@ -42,6 +43,7 @@ import org.apache.nifi.stream.io.DataOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Tags({"distributed", "cache", "state", "map", "cluster"})
 @SeeAlso(classNames = {"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
 @CapabilityDescription("Provides the ability to communicate with a DistributedMapCacheServer. This can be used in order to share a Map "
         + "between nodes in a NiFi cluster")

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
index 63d59ca..8c95c77 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
@@ -42,6 +43,7 @@ import org.apache.nifi.stream.io.DataOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@Tags({"distributed", "cache", "state", "set", "cluster"})
 @SeeAlso(classNames = {"org.apache.nifi.distributed.cache.server.DistributedSetCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
 @CapabilityDescription("Provides the ability to communicate with a DistributedSetCacheServer. This can be used in order to share a Set "
         + "between nodes in a NiFi cluster")

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html
deleted file mode 100644
index 1568635..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html
+++ /dev/null
@@ -1,45 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      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.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>Distributed Map Cache Client Service</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-        <p>
-            Below is an example of how to create a client connection to your distributed map cache server. 
-            Note that the identifier in this example is <code>cache-client</code>. If you are using this template
-            to create your own MapCacheClient service, replace the values in this template with values that are
-            suitable for your system. Possible options for <code>Server Hostname</code>, <code>Server Port</code>,
-            <code>Communications Timeout</code>, and <span style="font-style: italic;">SSL Context Service</span>.
-        </p>
-
-        <pre>
-&lt;?xml version="1.0" encoding="UTF-8" ?&gt;
-&lt;services&gt;
-    &lt;service&gt;
-        &lt;identifier&gt;cache-client&lt;/identifier&gt;
-        &lt;class&gt;org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService&lt;/class&gt;
-        &lt;property name="Server Hostname"&gt;localhost&lt;/property&gt;
-        &lt;property name="Server Port"&gt;4557&lt;/property&gt;
-        &lt;property name="Communications Timeout"&gt;30 secs&lt;/property&gt;
-    &lt;/service&gt;
-&lt;/services&gt;
-        </pre>
-    </body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html
deleted file mode 100644
index 740abec..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html
+++ /dev/null
@@ -1,46 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      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.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>Distributed Map Cache Client Service</title>
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-        <p>
-            Below is an example of how to create a distributed map cache server for clients to connect to.
-            Note that the identifier in this example is <code>cache-server</code>. If you are using this template
-            to create your own DistributedMapCache server, replace the values in this template with values that are
-            suitable for your system. Possible options for <code>Port</code>, <code>Maximum Cache Entries</code>,
-            <code>Eviction Strategy</code>, <span style="font-style: italic;">SSL Context Service</span>, and
-            <span style="font-style: italic;">Persistence Directory</span>
-        </p>
-
-        <pre>
-&lt;?xml version="1.0" encoding="UTF-8" ?&gt;
-&lt;services&gt;
-    &lt;service&gt;
-        &lt;identifier&gt;cache-server&lt;/identifier&gt;
-        &lt;class&gt;org.apache.nifi.distributed.cache.client.DistributedMapCacheServer&lt;/class&gt;
-        &lt;property name="Port"&gt;4557&lt;/property&gt;
-        &lt;property name="Maximum Cache Entries"&gt;10000&lt;/property&gt;
-        &lt;property name="Eviction Strategy"&gt;Least Recently Used&lt;/property&gt;
-    &lt;/service&gt;
-&lt;/services&gt;
-        </pre>
-    </body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/resources/docs/org.apache.nifi.http.StandardHttpContextMap/index.html
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/resources/docs/org.apache.nifi.http.StandardHttpContextMap/index.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/resources/docs/org.apache.nifi.http.StandardHttpContextMap/index.html
deleted file mode 100644
index 774c3d9..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/resources/docs/org.apache.nifi.http.StandardHttpContextMap/index.html
+++ /dev/null
@@ -1,67 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      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.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>StandardHttpContextMap</title>
-
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-        <h2>Description:</h2>
-        <p>
-            This is the standard implementation of the SSL Context Map. This service is used to provide
-            coordination between 
-            <a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a>
-            and
-            <a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a>
-            Processors.
-        </p>
-
-        <!-- Service Documentation ================================================== -->
-        <h2>Configuring the HTTP Context Map:</h2>
-        <p>
-            The <code>controller-services.xml</code> file is located in the NiFi <code>conf</code> 
-            directory. The user may set up any number of controller services within this file.
-        </p>
-
-        <p>
-            This controller service exposes a single property named <code>Maximum Outstanding Requests</code>.
-            This property determines the maximum number of HTTP requests that can be outstanding at any one time. 
-            Any attempt to register an additional HTTP Request will cause an error. The default value is 5000.
-            Below is an example of the template for a StandardHttpContextMap controller service.
-        </p>
-
-        <pre>
-&lt;?xml version="1.0" encoding="UTF-8" ?&gt;
-&lt;services&gt;
-    &lt;service&gt;
-        &lt;identifier&gt;http-context-map&lt;/identifier&gt;
-        &lt;class&gt;org.apache.nifi.http.StandardHttpContextMap&lt;/class&gt;
-        &lt;property name="Maximum Outstanding Requests"&gt;5000&lt;/property&gt;
-    &lt;/service&gt;
-&lt;/services&gt;
-        </pre>
-
-        <p>
-            <strong>See Also:</strong><br />
-            <a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a><br />
-            <a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a><br />
-        </p>
-
-    </body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8c006237/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html
deleted file mode 100644
index 525337d..0000000
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html
+++ /dev/null
@@ -1,49 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-    <!--
-      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.
-    -->
-    <head>
-        <meta charset="utf-8" />
-        <title>StandardSSLContextService</title>
-
-        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
-    </head>
-
-    <body>
-        <p>
-            Below is an example of the template for a SSLContext controller service. Note that the identifier 
-            in this example is <code>ssl-context</code>. If using this template to create your own SSLContext controller 
-            service, replace the property values with values that are suitable for your system. Possible options for 
-            <code>Keystore Type</code> and <code>Truststore Type</code> are <span style="font-style: italic;">JKS</span>
-            or <span style="font-style: italic;">PKCS12</span>.
-        </p>
-
-        <pre>
-&lt;?xml version="1.0" encoding="UTF-8" ?&gt;
-&lt;services&gt;
-    &lt;service&gt;
-        &lt;identifier&gt;ssl-context&lt;/identifier&gt;
-        &lt;class&gt;org.apache.nifi.ssl.StandardSSLContextService&lt;/class&gt;
-        &lt;property name="Keystore Filename"&gt;C:/testpki/localtest-ks.jks&lt;/property&gt;
-        &lt;property name="Keystore Password"&gt;localtest&lt;/property&gt;
-        &lt;property name="Keystore Type"&gt;JKS&lt;/property&gt;
-        &lt;property name="Truststore Filename"&gt;C:/testpki/localtest-ts.jks&lt;/property&gt;
-        &lt;property name="Truststore Password"&gt;localtest&lt;/property&gt;
-        &lt;property name="Truststore Type"&gt;JKS&lt;/property&gt;
-    &lt;/service&gt;
-&lt;/services&gt;
-        </pre>
-    </body>
-</html>


[3/7] incubator-nifi git commit: NIFI-533: Fixed code to conform to checkstyle

Posted by ma...@apache.org.
NIFI-533: Fixed code to conform to checkstyle


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

Branch: refs/heads/ListHDFS
Commit: dc7f7a82a68e7e19d0aeb52bd9bca44c277f4fba
Parents: 94945a6 384b2ac
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Apr 27 14:51:51 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Apr 27 14:51:51 2015 -0400

----------------------------------------------------------------------
 nifi-nar-maven-plugin/pom.xml                   |  181 +-
 .../src/main/java/org/apache/nifi/NarMojo.java  |    4 +-
 nifi-parent/DISCLAIMER                          |   15 +
 nifi-parent/LICENSE                             |  202 +++
 nifi-parent/NOTICE                              |    5 +
 nifi-parent/README.md                           |   65 +
 nifi-parent/pom.xml                             |  392 +++++
 nifi/LICENSE                                    |   22 +
 .../annotation/behavior/DynamicProperties.java  |    9 +-
 .../annotation/behavior/DynamicProperty.java    |   29 +-
 .../behavior/DynamicRelationship.java           |   15 +-
 .../annotation/behavior/ReadsAttribute.java     |   19 +-
 .../annotation/behavior/ReadsAttributes.java    |   13 +-
 .../annotation/behavior/SideEffectFree.java     |   23 +-
 .../annotation/behavior/TriggerSerially.java    |    9 +-
 .../TriggerWhenAnyDestinationAvailable.java     |    9 +-
 .../annotation/behavior/TriggerWhenEmpty.java   |   14 +-
 .../annotation/behavior/WritesAttribute.java    |   20 +-
 .../annotation/behavior/WritesAttributes.java   |   13 +-
 .../documentation/CapabilityDescription.java    |    7 +-
 .../nifi/annotation/documentation/SeeAlso.java  |   21 +-
 .../nifi/annotation/documentation/Tags.java     |   14 +-
 .../nifi/annotation/lifecycle/OnAdded.java      |   24 +-
 .../nifi/annotation/lifecycle/OnDisabled.java   |   35 +-
 .../nifi/annotation/lifecycle/OnEnabled.java    |   43 +-
 .../nifi/annotation/lifecycle/OnRemoved.java    |   27 +-
 .../nifi/annotation/lifecycle/OnScheduled.java  |   44 +-
 .../nifi/annotation/lifecycle/OnShutdown.java   |   23 +-
 .../nifi/annotation/lifecycle/OnStopped.java    |   24 +-
 .../annotation/lifecycle/OnUnscheduled.java     |   31 +-
 .../apache/nifi/authorization/Authority.java    |   12 +-
 .../nifi/authorization/AuthorityProvider.java   |   99 +-
 .../AuthorityProviderConfigurationContext.java  |   12 +-
 .../authorization/DownloadAuthorization.java    |   33 +-
 .../AbstractConfigurableComponent.java          |   14 +-
 .../apache/nifi/components/AllowableValue.java  |   30 +-
 .../nifi/components/ConfigurableComponent.java  |   16 +-
 .../nifi/components/PropertyDescriptor.java     |   79 +-
 .../apache/nifi/components/PropertyValue.java   |   40 +-
 .../nifi/components/ValidationContext.java      |   73 +-
 .../nifi/components/ValidationResult.java       |   16 +-
 .../controller/AbstractControllerService.java   |   26 +-
 .../nifi/controller/ConfigurationContext.java   |   10 +-
 .../nifi/controller/ControllerService.java      |   48 +-
 .../ControllerServiceInitializationContext.java |   16 +-
 .../controller/ControllerServiceLookup.java     |   49 +-
 .../apache/nifi/controller/FlowFileQueue.java   |   36 +-
 .../org/apache/nifi/controller/Snippet.java     |   48 +-
 .../org/apache/nifi/controller/Triggerable.java |   15 +-
 .../controller/annotation/OnConfigured.java     |    5 +-
 .../repository/ContentRepository.java           |  109 +-
 .../controller/repository/FlowFileRecord.java   |   14 +-
 .../repository/FlowFileRepository.java          |   27 +-
 .../repository/FlowFileSwapManager.java         |    6 +-
 .../controller/repository/QueueProvider.java    |    4 +-
 .../controller/repository/RepositoryRecord.java |   36 +-
 .../repository/claim/ContentClaim.java          |    6 +-
 .../repository/claim/ContentClaimManager.java   |   45 +-
 .../controller/status/ProcessGroupStatus.java   |   44 +-
 .../history/ComponentStatusRepository.java      |   58 +-
 .../status/history/MetricDescriptor.java        |   25 +-
 .../status/history/StatusHistory.java           |   14 +-
 .../status/history/StatusSnapshot.java          |   12 +-
 .../nifi/expression/AttributeExpression.java    |   30 +-
 .../expression/AttributeValueDecorator.java     |    4 +-
 .../expression/ExpressionLanguageCompiler.java  |   10 +-
 .../java/org/apache/nifi/flowfile/FlowFile.java |   16 +-
 .../org/apache/nifi/logging/ComponentLog.java   |   50 +-
 .../org/apache/nifi/logging/ProcessorLog.java   |   14 +-
 .../AbstractSessionFactoryProcessor.java        |   11 +-
 .../org/apache/nifi/processor/DataUnit.java     |    3 -
 .../apache/nifi/processor/FlowFileFilter.java   |    6 +-
 .../apache/nifi/processor/ProcessContext.java   |   31 +-
 .../apache/nifi/processor/ProcessSession.java   |  119 +-
 .../org/apache/nifi/processor/Processor.java    |    2 +-
 .../ProcessorInitializationContext.java         |   12 +-
 .../annotation/CapabilityDescription.java       |    3 +-
 .../nifi/processor/annotation/EventDriven.java  |    4 +-
 .../nifi/processor/annotation/OnAdded.java      |    3 +-
 .../nifi/processor/annotation/OnRemoved.java    |    3 +-
 .../nifi/processor/annotation/OnScheduled.java  |    5 +-
 .../nifi/processor/annotation/OnShutdown.java   |    3 +-
 .../nifi/processor/annotation/OnStopped.java    |    3 +-
 .../processor/annotation/OnUnscheduled.java     |    3 +-
 .../processor/annotation/SideEffectFree.java    |    3 +-
 .../processor/annotation/SupportsBatching.java  |    3 +-
 .../apache/nifi/processor/annotation/Tags.java  |    8 +-
 .../processor/annotation/TriggerSerially.java   |    5 +-
 .../TriggerWhenAnyDestinationAvailable.java     |    4 +-
 .../processor/annotation/TriggerWhenEmpty.java  |    3 +-
 .../nifi/processor/io/InputStreamCallback.java  |    4 +-
 .../nifi/processor/io/OutputStreamCallback.java |    4 +-
 .../nifi/processor/io/StreamCallback.java       |   10 +-
 .../nifi/provenance/ProvenanceEventBuilder.java |  120 +-
 .../nifi/provenance/ProvenanceEventRecord.java  |  137 +-
 .../provenance/ProvenanceEventRepository.java   |   66 +-
 .../lineage/ComputeLineageResult.java           |   26 +-
 .../lineage/ComputeLineageSubmission.java       |   28 +-
 .../apache/nifi/provenance/lineage/Lineage.java |    8 +-
 .../nifi/provenance/lineage/LineageNode.java    |   22 +-
 .../apache/nifi/provenance/search/Query.java    |    7 +-
 .../nifi/provenance/search/QueryResult.java     |   30 +-
 .../nifi/provenance/search/QuerySubmission.java |   18 +-
 .../nifi/provenance/search/SearchableField.java |   20 +-
 .../apache/nifi/remote/RemoteDestination.java   |   47 +-
 .../nifi/reporting/AbstractReportingTask.java   |   26 +-
 .../nifi/reporting/BulletinRepository.java      |   32 +-
 .../org/apache/nifi/reporting/EventAccess.java  |   12 +-
 .../apache/nifi/reporting/ReportingContext.java |   36 +-
 .../ReportingInitializationContext.java         |   33 +-
 .../apache/nifi/reporting/ReportingTask.java    |    6 +-
 .../org/apache/nifi/search/SearchContext.java   |   18 +-
 .../org/apache/nifi/search/SearchResult.java    |   16 +-
 .../org/apache/nifi/web/ComponentDetails.java   |   35 +-
 .../apache/nifi/web/ConfigurationAction.java    |   30 +-
 .../nifi/web/NiFiWebConfigurationContext.java   |   68 +-
 .../web/NiFiWebConfigurationRequestContext.java |    5 +-
 .../org/apache/nifi/web/NiFiWebContext.java     |   18 +-
 .../apache/nifi/web/NiFiWebRequestContext.java  |    8 +-
 .../nifi/web/ProcessorConfigurationAction.java  |   24 +-
 .../main/java/org/apache/nifi/web/Revision.java |    4 +-
 .../org/apache/nifi/web/UiExtensionType.java    |   11 +-
 .../org/apache/nifi/web/ViewableContent.java    |   41 +-
 .../nifi/components/TestPropertyDescriptor.java |   18 +-
 nifi/nifi-assembly/LICENSE                      |   22 +
 nifi/nifi-assembly/NOTICE                       |   35 +-
 nifi/nifi-assembly/pom.xml                      |  928 +++++-----
 .../apache/nifi/bootstrap/BootstrapCodec.java   |  118 +-
 .../org/apache/nifi/bootstrap/NiFiListener.java |  196 +--
 .../java/org/apache/nifi/bootstrap/RunNiFi.java | 1641 +++++++++---------
 .../org/apache/nifi/bootstrap/ShutdownHook.java |  120 +-
 .../exception/InvalidCommandException.java      |   33 +-
 .../nifi/provenance/AsyncLineageSubmission.java |    1 +
 .../nifi/provenance/AsyncQuerySubmission.java   |    7 +-
 .../nifi/provenance/SearchableFields.java       |   21 +-
 .../nifi/provenance/StandardLineageResult.java  |    8 +-
 .../nifi-expression-language/pom.xml            |    7 +
 .../language/InvalidPreparedQuery.java          |   13 +-
 .../attribute/expression/language/Query.java    |   81 +-
 .../functions/GreaterThanEvaluator.java         |    3 -
 .../functions/GreaterThanOrEqualEvaluator.java  |    3 -
 .../evaluation/functions/IsEmptyEvaluator.java  |    5 +-
 .../evaluation/functions/LessThanEvaluator.java |    3 -
 .../functions/LessThanOrEqualEvaluator.java     |    3 -
 .../functions/ReplaceEmptyEvaluator.java        |    7 +-
 .../evaluation/reduce/CountEvaluator.java       |   10 +-
 .../evaluation/reduce/JoinEvaluator.java        |   17 +-
 .../evaluation/selection/MappingEvaluator.java  |    7 +-
 .../selection/MultiMatchAttributeEvaluator.java |    2 +-
 .../exception/IllegalAttributeException.java    |    1 +
 .../expression/language/TestQuery.java          |  385 ++--
 .../language/TestStandardPreparedQuery.java     |   26 +-
 .../apache/nifi/util/FlowFilePackagerV1.java    |    2 +-
 .../apache/nifi/util/FlowFileUnpackagerV1.java  |    2 +-
 .../nifi-hl7-query-language/pom.xml             |  204 +--
 .../org/apache/nifi/hl7/hapi/EmptyField.java    |   16 +-
 .../org/apache/nifi/hl7/hapi/HapiField.java     |   95 +-
 .../org/apache/nifi/hl7/hapi/HapiMessage.java   |  115 +-
 .../org/apache/nifi/hl7/hapi/HapiSegment.java   |   70 +-
 .../apache/nifi/hl7/hapi/SingleValueField.java  |   31 +-
 .../java/org/apache/nifi/hl7/io/HL7Reader.java  |    4 +-
 .../hl7/io/exception/InvalidHL7Exception.java   |   29 +-
 .../org/apache/nifi/hl7/model/HL7Component.java |    6 +-
 .../org/apache/nifi/hl7/model/HL7Field.java     |    1 -
 .../org/apache/nifi/hl7/model/HL7Message.java   |    8 +-
 .../org/apache/nifi/hl7/model/HL7Segment.java   |    8 +-
 .../org/apache/nifi/hl7/query/Declaration.java  |   12 +-
 .../org/apache/nifi/hl7/query/HL7Query.java     |  646 ++++---
 .../org/apache/nifi/hl7/query/QueryResult.java  |   15 +-
 .../org/apache/nifi/hl7/query/ResultHit.java    |    7 +-
 .../org/apache/nifi/hl7/query/Selection.java    |   31 +-
 .../hl7/query/evaluator/BooleanEvaluator.java   |    6 +-
 .../nifi/hl7/query/evaluator/Evaluator.java     |   11 +-
 .../hl7/query/evaluator/IntegerEvaluator.java   |    7 +-
 .../hl7/query/evaluator/StringEvaluator.java    |    8 +-
 .../comparison/AbstractComparisonEvaluator.java |  163 +-
 .../comparison/AbstractNumericComparison.java   |   85 +-
 .../evaluator/comparison/EqualsEvaluator.java   |   16 +-
 .../comparison/GreaterThanEvaluator.java        |   16 +-
 .../comparison/GreaterThanOrEqualEvaluator.java |   16 +-
 .../evaluator/comparison/IsNullEvaluator.java   |   83 +-
 .../evaluator/comparison/LessThanEvaluator.java |   15 +-
 .../comparison/LessThanOrEqualEvaluator.java    |   15 +-
 .../comparison/NotEqualsEvaluator.java          |   16 +-
 .../evaluator/comparison/NotEvaluator.java      |   23 +-
 .../evaluator/comparison/NotNullEvaluator.java  |   75 +-
 .../literal/IntegerLiteralEvaluator.java        |   22 +-
 .../literal/StringLiteralEvaluator.java         |   21 +-
 .../hl7/query/evaluator/logic/AndEvaluator.java |   37 +-
 .../hl7/query/evaluator/logic/OrEvaluator.java  |   37 +-
 .../message/DeclaredReferenceEvaluator.java     |   33 +-
 .../query/evaluator/message/DotEvaluator.java   |  111 +-
 .../query/evaluator/message/FieldEvaluator.java |   69 +-
 .../evaluator/message/MessageEvaluator.java     |   12 +-
 .../evaluator/message/SegmentEvaluator.java     |   41 +-
 .../exception/HL7QueryParsingException.java     |   33 +-
 .../nifi/hl7/query/result/MissedResult.java     |   59 +-
 .../hl7/query/result/StandardQueryResult.java   |   75 +-
 .../hl7/query/result/StandardResultHit.java     |   31 +-
 .../org/apache/nifi/hl7/query/TestHL7Query.java |  524 +++---
 .../nifi/processor/util/FlowFileFilters.java    |    2 +-
 .../nifi/processor/util/SSLProperties.java      |    8 +-
 .../nifi/processor/util/StandardValidators.java |  140 +-
 .../processor/util/TestStandardValidators.java  |   18 +-
 .../org/apache/nifi/util/NiFiProperties.java    |   14 +-
 .../nifi/security/util/CertificateUtils.java    |   21 +-
 .../nifi/security/util/SecurityStoreTypes.java  |    8 +-
 .../nifi/security/util/SslContextFactory.java   |   36 +-
 .../nifi-site-to-site-client/pom.xml            |   64 +-
 .../remote/AbstractCommunicationsSession.java   |    9 +-
 .../org/apache/nifi/remote/Communicant.java     |   23 +-
 .../main/java/org/apache/nifi/remote/Peer.java  |   20 +-
 .../org/apache/nifi/remote/PeerDescription.java |    7 +-
 .../java/org/apache/nifi/remote/PeerStatus.java |    7 +-
 .../nifi/remote/RemoteResourceInitiator.java    |   48 +-
 .../org/apache/nifi/remote/Transaction.java     |  295 ++--
 .../nifi/remote/TransactionCompletion.java      |   49 +-
 .../apache/nifi/remote/TransferDirection.java   |   13 +-
 .../nifi/remote/client/SiteToSiteClient.java    |  831 ++++-----
 .../remote/client/SiteToSiteClientConfig.java   |  170 +-
 .../client/socket/EndpointConnection.java       |   15 +-
 .../client/socket/EndpointConnectionPool.java   |  627 ++++---
 .../nifi/remote/client/socket/SocketClient.java |  319 ++--
 .../apache/nifi/remote/codec/FlowFileCodec.java |   15 +-
 .../remote/codec/StandardFlowFileCodec.java     |   33 +-
 .../remote/exception/HandshakeException.java    |    9 +-
 .../exception/PortNotRunningException.java      |    7 +-
 .../remote/exception/ProtocolException.java     |    4 +-
 .../remote/exception/UnknownPortException.java  |    6 +-
 .../SocketChannelCommunicationsSession.java     |   23 +-
 .../remote/io/socket/SocketChannelInput.java    |   11 +-
 .../remote/io/socket/SocketChannelOutput.java   |   11 +-
 .../SSLSocketChannelCommunicationsSession.java  |   23 +-
 .../io/socket/ssl/SSLSocketChannelInput.java    |    9 +-
 .../io/socket/ssl/SSLSocketChannelOutput.java   |    5 +-
 .../nifi/remote/protocol/ClientProtocol.java    |   18 +-
 .../remote/protocol/CommunicationsInput.java    |    5 +-
 .../remote/protocol/CommunicationsSession.java  |    6 +-
 .../apache/nifi/remote/protocol/DataPacket.java |   33 +-
 .../protocol/socket/HandshakeProperty.java      |   38 +-
 .../nifi/remote/protocol/socket/Response.java   |   11 +-
 .../remote/protocol/socket/ResponseCode.java    |   65 +-
 .../protocol/socket/SocketClientProtocol.java   |  388 +++--
 .../socket/SocketClientTransaction.java         |  468 ++---
 .../SocketClientTransactionCompletion.java      |    2 +-
 .../nifi/remote/util/NiFiRestApiUtil.java       |   24 +-
 .../nifi/remote/util/PeerStatusCache.java       |    3 +-
 .../nifi/remote/util/StandardDataPacket.java    |   44 +-
 .../socket/TestEndpointConnectionStatePool.java |   17 +-
 .../client/socket/TestSiteToSiteClient.java     |   42 +-
 .../nifi/io/nio/AbstractChannelReader.java      |   10 +-
 .../java/org/apache/nifi/io/nio/BufferPool.java |   10 +-
 .../apache/nifi/io/nio/ChannelDispatcher.java   |   14 +-
 .../org/apache/nifi/io/nio/ChannelListener.java |    7 +-
 .../nifi/io/nio/DatagramChannelReader.java      |   12 +-
 .../apache/nifi/io/nio/SocketChannelReader.java |   12 +-
 .../nifi/io/nio/consumer/StreamConsumer.java    |   12 +-
 .../nifi/io/socket/SSLContextFactory.java       |   12 +-
 .../io/socket/ServerSocketConfiguration.java    |    6 +-
 .../nifi/io/socket/SocketConfiguration.java     |    6 +-
 .../org/apache/nifi/io/socket/SocketUtils.java  |    6 +-
 .../io/socket/multicast/MulticastListener.java  |    5 +-
 .../apache/nifi/io/nio/example/TCPClient.java   |    3 +-
 .../flowfile/attributes/CoreAttributes.java     |   27 +-
 .../attributes/FlowFileAttributeKey.java        |    1 +
 .../apache/nifi/remote/VersionNegotiator.java   |   10 +-
 .../nifi/remote/io/CompressionInputStream.java  |    5 +-
 .../nifi/remote/io/CompressionOutputStream.java |    2 +-
 .../remote/io/socket/BufferStateManager.java    |    2 +-
 .../io/socket/SocketChannelInputStream.java     |    9 +-
 .../io/socket/SocketChannelOutputStream.java    |    3 +-
 .../remote/io/socket/ssl/SSLSocketChannel.java  |    4 +-
 .../socket/ssl/SSLSocketChannelInputStream.java |    2 +-
 .../nifi/stream/io/ByteArrayOutputStream.java   |    6 +-
 .../nifi/stream/io/ByteCountingInputStream.java |    5 +
 .../stream/io/ByteCountingOutputStream.java     |    6 +
 .../stream/io/LeakyBucketStreamThrottler.java   |    8 +-
 .../nifi/stream/io/LimitingInputStream.java     |    2 +-
 .../stream/io/MinimumLengthInputStream.java     |  129 +-
 .../org/apache/nifi/stream/io/StreamUtils.java  |   52 +-
 .../io/util/NonThreadSafeCircularBuffer.java    |    2 +-
 .../java/org/apache/nifi/util/EscapeUtils.java  |   15 +-
 .../java/org/apache/nifi/util/FormatUtils.java  |   16 +-
 .../apache/nifi/util/NaiveSearchRingBuffer.java |   14 +-
 .../java/org/apache/nifi/util/ObjectHolder.java |    1 -
 .../java/org/apache/nifi/util/RingBuffer.java   |   19 +-
 .../java/org/apache/nifi/util/StopWatch.java    |    8 +-
 .../main/java/org/apache/nifi/util/Tuple.java   |    6 -
 .../concurrency/DebugDisabledTimedLock.java     |   11 +-
 .../util/concurrency/DebugEnabledTimedLock.java |   16 +-
 .../org/apache/nifi/util/file/FileUtils.java    |   57 +-
 .../file/monitor/SynchronousFileWatcher.java    |    4 +-
 .../org/apache/nifi/util/search/Search.java     |    6 +-
 .../org/apache/nifi/util/search/SearchTerm.java |   14 +-
 .../file/monitor/TestCompoundUpdateMonitor.java |    1 -
 .../monitor/TestSynchronousFileWatcher.java     |    1 -
 .../nifi/util/timebuffer/TestRingBuffer.java    |    6 +-
 .../org/apache/nifi/web/util/ClientUtils.java   |   38 +-
 .../org/wali/MinimalLockingWriteAheadLog.java   |   70 +-
 .../src/main/java/org/wali/SerDe.java           |   40 +-
 .../java/org/wali/WriteAheadRepository.java     |   16 +-
 nifi/nifi-docs/pom.xml                          |    9 +
 nifi/nifi-external/nifi-spark-receiver/pom.xml  |   22 +-
 .../org/apache/nifi/spark/NiFiDataPacket.java   |   23 +-
 .../org/apache/nifi/spark/NiFiReceiver.java     |  236 +--
 ...kControllerServiceInitializationContext.java |    8 +-
 .../nifi/util/MockControllerServiceLookup.java  |   14 +-
 .../java/org/apache/nifi/util/MockFlowFile.java |   20 +-
 .../apache/nifi/util/MockProcessContext.java    |   15 +-
 .../apache/nifi/util/MockProcessSession.java    |   33 +-
 .../MockProcessorInitializationContext.java     |    4 +-
 .../org/apache/nifi/util/MockProcessorLog.java  |  124 --
 .../nifi/util/MockProvenanceReporter.java       |    4 +-
 .../apache/nifi/util/MockReportingContext.java  |    6 +-
 .../MockReportingInitializationContext.java     |    2 +-
 .../apache/nifi/util/MockValidationContext.java |   20 +-
 .../org/apache/nifi/util/ReflectionUtils.java   |   21 +-
 .../apache/nifi/util/SharedSessionState.java    |    1 -
 .../nifi/util/StandardProcessorTestRunner.java  |  123 +-
 .../java/org/apache/nifi/util/TestRunner.java   |  428 ++---
 .../util/TestStandardProcessorTestRunner.java   |   30 +-
 .../src/main/resources/META-INF/NOTICE          |   74 +
 .../nifi-aws-bundle/nifi-aws-processors/pom.xml |   19 +-
 .../processors/aws/AbstractAWSProcessor.java    |   99 +-
 .../processors/aws/s3/AbstractS3Processor.java  |  152 +-
 .../nifi/processors/aws/s3/FetchS3Object.java   |   78 +-
 .../nifi/processors/aws/s3/PutS3Object.java     |  100 +-
 .../aws/sns/AbstractSNSProcessor.java           |   42 +-
 .../apache/nifi/processors/aws/sns/PutSNS.java  |   98 +-
 .../aws/sqs/AbstractSQSProcessor.java           |   24 +-
 .../nifi/processors/aws/sqs/DeleteSQS.java      |   40 +-
 .../apache/nifi/processors/aws/sqs/GetSQS.java  |  137 +-
 .../apache/nifi/processors/aws/sqs/PutSQS.java  |   69 +-
 .../processors/aws/s3/TestFetchS3Object.java    |   31 +-
 .../nifi/processors/aws/s3/TestPutS3Object.java |   45 +-
 .../nifi/processors/aws/sns/TestPutSNS.java     |   25 +-
 .../nifi/processors/aws/sqs/TestGetSQS.java     |   23 +-
 .../nifi/processors/aws/sqs/TestPutSQS.java     |   23 +-
 nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml   |   18 +-
 .../nifi-framework/nifi-administration/pom.xml  |    8 +
 .../org/apache/nifi/admin/RepositoryUtils.java  |    7 +-
 .../nifi/admin/UserDataSourceFactoryBean.java   |   12 -
 .../org/apache/nifi/admin/dao/ActionDAO.java    |   28 +-
 .../org/apache/nifi/admin/dao/AuthorityDAO.java |   19 +-
 .../java/org/apache/nifi/admin/dao/UserDAO.java |   52 +-
 .../nifi/admin/dao/impl/StandardActionDAO.java  |   93 +-
 .../apache/nifi/admin/service/AuditService.java |   28 +-
 .../apache/nifi/admin/service/UserService.java  |   70 +-
 .../service/action/AbstractUserAction.java      |   18 +-
 .../service/action/AdministrationAction.java    |    8 +-
 .../service/action/AuthorizeUserAction.java     |    6 +-
 .../admin/service/action/DeleteUserAction.java  |    2 +-
 .../service/action/FindUserByIdAction.java      |    5 +-
 .../service/action/SeedUserAccountsAction.java  |    4 +-
 .../service/impl/StandardAuditService.java      |   11 -
 .../admin/service/impl/StandardUserService.java |   29 +-
 .../admin/service/transaction/Transaction.java  |    6 +-
 .../AuthorityProviderFactoryBean.java           |   47 +-
 .../org/apache/nifi/user/AccountStatus.java     |    4 +-
 .../java/org/apache/nifi/user/NiFiUser.java     |    2 +-
 .../service/action/AuthorizeUserActionTest.java |   22 +-
 .../service/action/CreateUserActionTest.java    |   14 +-
 .../service/action/DisableUserActionTest.java   |   11 +-
 .../action/InvalidateUserAccountActionTest.java |    9 +-
 .../action/RequestUserAccountActionTest.java    |    6 +-
 .../action/SeedUserAccountsActionTest.java      |    3 +-
 .../action/SetUserAuthoritiesActionTest.java    |    6 +-
 .../nifi/web/api/dto/BulletinBoardDTO.java      |    8 +-
 .../apache/nifi/web/api/dto/BulletinDTO.java    |   38 +-
 .../nifi/web/api/dto/BulletinQueryDTO.java      |   24 +-
 .../org/apache/nifi/web/api/dto/ClusterDTO.java |    8 +-
 .../nifi/web/api/dto/ComponentHistoryDTO.java   |    8 +-
 .../apache/nifi/web/api/dto/ConnectableDTO.java |   36 +-
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |   24 +-
 .../web/api/dto/ControllerConfigurationDTO.java |   44 +-
 .../apache/nifi/web/api/dto/ControllerDTO.java  |   56 +-
 .../nifi/web/api/dto/ControllerServiceDTO.java  |   68 +-
 ...ontrollerServiceReferencingComponentDTO.java |   69 +-
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   16 +-
 .../apache/nifi/web/api/dto/CountersDTO.java    |    8 +-
 .../nifi/web/api/dto/DocumentedTypeDTO.java     |   12 +-
 .../apache/nifi/web/api/dto/FlowSnippetDTO.java |   37 +-
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   14 +-
 .../nifi/web/api/dto/NiFiComponentDTO.java      |    5 +-
 .../org/apache/nifi/web/api/dto/NodeDTO.java    |   40 +-
 .../apache/nifi/web/api/dto/NodeEventDTO.java   |   12 +-
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |    8 +-
 .../org/apache/nifi/web/api/dto/PortDTO.java    |   34 +-
 .../apache/nifi/web/api/dto/PositionDTO.java    |    8 +-
 .../nifi/web/api/dto/PreviousValueDTO.java      |   12 +-
 .../nifi/web/api/dto/ProcessGroupDTO.java       |   46 +-
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |   59 +-
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |   22 +-
 .../nifi/web/api/dto/PropertyDescriptorDTO.java |   63 +-
 .../nifi/web/api/dto/PropertyHistoryDTO.java    |    4 +-
 .../nifi/web/api/dto/RelationshipDTO.java       |   12 +-
 .../api/dto/RemoteProcessGroupContentsDTO.java  |    8 +-
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |   70 +-
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |   42 +-
 .../nifi/web/api/dto/ReportingTaskDTO.java      |  272 ++-
 .../apache/nifi/web/api/dto/RevisionDTO.java    |    5 +-
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |   68 +-
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  116 +-
 .../apache/nifi/web/api/dto/TemplateDTO.java    |   24 +-
 .../org/apache/nifi/web/api/dto/UserDTO.java    |   42 +-
 .../apache/nifi/web/api/dto/UserGroupDTO.java   |   16 +-
 .../nifi/web/api/dto/action/ActionDTO.java      |   40 +-
 .../nifi/web/api/dto/action/HistoryDTO.java     |   12 +-
 .../web/api/dto/action/HistoryQueryDTO.java     |   32 +-
 .../component/details/ExtensionDetailsDTO.java  |    4 +-
 .../details/RemoteProcessGroupDetailsDTO.java   |    4 +-
 .../dto/action/details/ConfigureDetailsDTO.java |   12 +-
 .../dto/action/details/ConnectDetailsDTO.java   |   28 +-
 .../api/dto/action/details/MoveDetailsDTO.java  |   16 +-
 .../api/dto/action/details/PurgeDetailsDTO.java |    4 +-
 .../web/api/dto/provenance/AttributeDTO.java    |   12 +-
 .../web/api/dto/provenance/ProvenanceDTO.java   |   36 +-
 .../api/dto/provenance/ProvenanceEventDTO.java  |  186 +-
 .../dto/provenance/ProvenanceOptionsDTO.java    |    4 +-
 .../dto/provenance/ProvenanceRequestDTO.java    |   24 +-
 .../dto/provenance/ProvenanceResultsDTO.java    |   28 +-
 .../ProvenanceSearchableFieldDTO.java           |   16 +-
 .../api/dto/provenance/lineage/LineageDTO.java  |   36 +-
 .../provenance/lineage/LineageRequestDTO.java   |   12 +-
 .../provenance/lineage/LineageResultsDTO.java   |   12 +-
 .../provenance/lineage/ProvenanceLinkDTO.java   |   20 +-
 .../provenance/lineage/ProvenanceNodeDTO.java   |   36 +-
 .../dto/search/ComponentSearchResultDTO.java    |   16 +-
 .../web/api/dto/search/NodeSearchResultDTO.java |    8 +-
 .../web/api/dto/search/SearchResultsDTO.java    |   28 +-
 .../dto/search/UserGroupSearchResultDTO.java    |    4 +-
 .../web/api/dto/search/UserSearchResultDTO.java |    8 +-
 .../dto/status/ClusterConnectionStatusDTO.java  |   14 +-
 .../api/dto/status/ClusterPortStatusDTO.java    |   14 +-
 .../status/ClusterProcessGroupStatusDTO.java    |   89 +
 .../dto/status/ClusterProcessorStatusDTO.java   |   20 +-
 .../ClusterRemoteProcessGroupStatusDTO.java     |   14 +-
 .../web/api/dto/status/ClusterStatusDTO.java    |    4 +-
 .../api/dto/status/ClusterStatusHistoryDTO.java |   12 +-
 .../web/api/dto/status/ConnectionStatusDTO.java |   42 +-
 .../web/api/dto/status/ControllerStatusDTO.java |   42 +-
 .../api/dto/status/NodeConnectionStatusDTO.java |    8 +-
 .../web/api/dto/status/NodePortStatusDTO.java   |    8 +-
 .../dto/status/NodeProcessGroupStatusDTO.java   |   57 +
 .../api/dto/status/NodeProcessorStatusDTO.java  |    8 +-
 .../status/NodeRemoteProcessGroupStatusDTO.java |    8 +-
 .../nifi/web/api/dto/status/NodeStatusDTO.java  |    8 +-
 .../api/dto/status/NodeStatusHistoryDTO.java    |    8 +-
 .../nifi/web/api/dto/status/PortStatusDTO.java  |   36 +-
 .../api/dto/status/ProcessGroupStatusDTO.java   |   81 +-
 .../web/api/dto/status/ProcessorStatusDTO.java  |   46 +-
 .../web/api/dto/status/RemotePortStatusDTO.java |   20 +-
 .../dto/status/RemoteProcessGroupStatusDTO.java |   38 +-
 .../nifi/web/api/dto/status/StatusDTO.java      |    4 +-
 .../web/api/dto/status/StatusDescriptorDTO.java |   16 +-
 .../web/api/dto/status/StatusHistoryDTO.java    |   16 +-
 .../api/dto/status/StatusHistoryDetailDTO.java  |    8 +-
 .../web/api/dto/status/StatusSnapshotDTO.java   |    8 +-
 .../nifi/web/api/entity/AuthorityEntity.java    |    8 +-
 .../entity/ClusterProcessGroupStatusEntity.java |   45 +
 .../api/entity/ClusterSearchResultsEntity.java  |    4 +-
 .../nifi/web/api/entity/ConnectionEntity.java   |    4 +-
 .../nifi/web/api/entity/ConnectionsEntity.java  |    4 +-
 .../web/api/entity/ControllerServiceEntity.java |    6 +-
 ...ollerServiceReferencingComponentsEntity.java |    5 +-
 .../entity/ControllerServiceTypesEntity.java    |    4 +-
 .../api/entity/ControllerServicesEntity.java    |    4 +-
 .../nifi/web/api/entity/CounterEntity.java      |    4 +-
 .../nifi/web/api/entity/CountersEntity.java     |    6 +-
 .../org/apache/nifi/web/api/entity/Entity.java  |    4 +-
 .../nifi/web/api/entity/FunnelsEntity.java      |    4 +-
 .../nifi/web/api/entity/InputPortEntity.java    |    4 +-
 .../nifi/web/api/entity/InputPortsEntity.java   |    4 +-
 .../nifi/web/api/entity/LabelsEntity.java       |    4 +-
 .../nifi/web/api/entity/OutputPortEntity.java   |    4 +-
 .../nifi/web/api/entity/OutputPortsEntity.java  |    4 +-
 .../web/api/entity/PrioritizerTypesEntity.java  |    4 +-
 .../web/api/entity/ProcessorTypesEntity.java    |    4 +-
 .../nifi/web/api/entity/ProcessorsEntity.java   |    4 +-
 .../api/entity/PropertyDescriptorEntity.java    |    1 -
 .../nifi/web/api/entity/ProvenanceEntity.java   |    4 +-
 .../api/entity/RemoteProcessGroupsEntity.java   |    4 +-
 .../web/api/entity/ReportingTaskEntity.java     |    6 +-
 .../api/entity/ReportingTaskTypesEntity.java    |    4 +-
 .../web/api/entity/ReportingTasksEntity.java    |    4 +-
 .../web/api/entity/SearchResultsEntity.java     |    4 +-
 .../nifi/web/api/entity/TemplatesEntity.java    |    4 +-
 .../web/api/entity/UserSearchResultsEntity.java |    8 +-
 .../apache/nifi/web/api/entity/UsersEntity.java |    4 +-
 .../apache/nifi/documentation/DocGenerator.java |   12 +-
 .../html/HtmlDocumentationWriter.java           |  231 +--
 .../html/HtmlProcessorDocumentationWriter.java  |   77 +-
 .../FullyDocumentedControllerService.java       |   51 +-
 .../example/FullyDocumentedProcessor.java       |  156 +-
 .../example/FullyDocumentedReportingTask.java   |   32 +-
 .../documentation/example/NakedProcessor.java   |    8 +-
 .../documentation/example/SampleService.java    |    2 +-
 .../html/HtmlDocumentationWriterTest.java       |   94 +-
 .../html/ProcessorDocumentationWriterTest.java  |  135 +-
 .../nifi/documentation/html/XmlValidator.java   |   29 +-
 .../protocol/ClusterManagerProtocolSender.java  |   22 +-
 .../cluster/protocol/ConnectionRequest.java     |    6 +-
 .../cluster/protocol/ConnectionResponse.java    |   54 +-
 .../apache/nifi/cluster/protocol/Heartbeat.java |   19 +-
 .../nifi/cluster/protocol/NodeBulletins.java    |    2 +-
 .../nifi/cluster/protocol/NodeIdentifier.java   |  102 +-
 .../cluster/protocol/NodeProtocolSender.java    |   43 +-
 .../nifi/cluster/protocol/ProtocolContext.java  |   12 +-
 .../cluster/protocol/ProtocolException.java     |    9 +-
 .../nifi/cluster/protocol/ProtocolHandler.java  |   17 +-
 .../nifi/cluster/protocol/ProtocolListener.java |   31 +-
 .../protocol/ProtocolMessageMarshaller.java     |    7 +-
 .../protocol/ProtocolMessageUnmarshaller.java   |   13 +-
 .../nifi/cluster/protocol/StandardDataFlow.java |   34 +-
 .../UnknownServiceAddressException.java         |    9 +-
 .../impl/ClusterManagerProtocolSenderImpl.java  |  101 +-
 .../ClusterManagerProtocolSenderListener.java   |   28 +-
 .../protocol/impl/ClusterServiceDiscovery.java  |    5 +-
 .../protocol/impl/ClusterServiceLocator.java    |  124 +-
 .../impl/ClusterServicesBroadcaster.java        |   86 +-
 .../protocol/impl/CopyingInputStream.java       |   25 +-
 .../impl/MulticastProtocolListener.java         |   66 +-
 .../protocol/impl/NodeProtocolSenderImpl.java   |   61 +-
 .../impl/NodeProtocolSenderListener.java        |   16 +-
 .../protocol/impl/SocketProtocolListener.java   |   52 +-
 .../protocol/jaxb/JaxbProtocolContext.java      |   21 +-
 .../jaxb/message/AdaptedConnectionRequest.java  |    7 +-
 .../jaxb/message/AdaptedConnectionResponse.java |   19 +-
 .../protocol/jaxb/message/AdaptedCounter.java   |   11 +-
 .../protocol/jaxb/message/AdaptedDataFlow.java  |    9 +-
 .../protocol/jaxb/message/AdaptedHeartbeat.java |   13 +-
 .../jaxb/message/AdaptedNodeBulletins.java      |   11 +-
 .../jaxb/message/AdaptedNodeIdentifier.java     |   13 +-
 .../jaxb/message/ConnectionRequestAdapter.java  |    4 +-
 .../jaxb/message/ConnectionResponseAdapter.java |   12 +-
 .../protocol/jaxb/message/DataFlowAdapter.java  |   10 +-
 .../protocol/jaxb/message/HeartbeatAdapter.java |   14 +-
 .../jaxb/message/JaxbProtocolUtils.java         |    6 +-
 .../jaxb/message/NodeBulletinsAdapter.java      |   10 +-
 .../jaxb/message/NodeIdentifierAdapter.java     |    6 +-
 .../protocol/jaxb/message/ObjectFactory.java    |   35 +-
 .../message/ConnectionRequestMessage.java       |   11 +-
 .../message/ConnectionResponseMessage.java      |   27 +-
 .../ControllerStartupFailureMessage.java        |    7 +-
 .../protocol/message/DisconnectMessage.java     |    8 +-
 .../protocol/message/ExceptionMessage.java      |    9 +-
 .../protocol/message/FlowRequestMessage.java    |    6 +-
 .../protocol/message/FlowResponseMessage.java   |    4 +-
 .../protocol/message/HeartbeatMessage.java      |    4 +-
 .../message/MulticastProtocolMessage.java       |   17 +-
 .../protocol/message/NodeBulletinsMessage.java  |    4 +-
 .../cluster/protocol/message/PingMessage.java   |   13 +-
 .../message/PrimaryRoleAssignmentMessage.java   |    4 +-
 .../protocol/message/ProtocolMessage.java       |   22 +-
 .../message/ReconnectionFailureMessage.java     |    8 +-
 .../message/ReconnectionRequestMessage.java     |   19 +-
 .../message/ReconnectionResponseMessage.java    |    2 +-
 .../message/ServiceBroadcastMessage.java        |    9 +-
 .../MulticastConfigurationFactoryBean.java      |   12 +-
 .../ServerSocketConfigurationFactoryBean.java   |   13 +-
 .../spring/SocketConfigurationFactoryBean.java  |   16 +-
 .../ClusterManagerProtocolSenderImplTest.java   |   58 +-
 .../impl/ClusterServiceDiscoveryTest.java       |   56 +-
 .../impl/ClusterServiceLocatorTest.java         |   64 +-
 .../impl/ClusterServicesBroadcasterTest.java    |   63 +-
 .../impl/MulticastProtocolListenerTest.java     |   59 +-
 .../impl/NodeProtocolSenderImplTest.java        |   83 +-
 .../testutils/ReflexiveProtocolHandler.java     |    6 +-
 .../nifi/cluster/context/ClusterContext.java    |   36 +-
 .../cluster/context/ClusterContextImpl.java     |   10 +-
 .../context/ClusterContextThreadLocal.java      |   12 +-
 .../cluster/firewall/ClusterNodeFirewall.java   |    5 +-
 .../impl/FileBasedClusterNodeFirewall.java      |    8 +-
 .../nifi/cluster/flow/ClusterDataFlow.java      |    8 +-
 .../apache/nifi/cluster/flow/DataFlowDao.java   |    2 +-
 .../cluster/flow/DataFlowManagementService.java |   23 +-
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |   52 +-
 .../impl/DataFlowManagementServiceImpl.java     |   38 +-
 .../nifi/cluster/manager/ClusterManager.java    |    8 +-
 .../nifi/cluster/manager/NodeResponse.java      |   18 +-
 .../cluster/manager/impl/WebClusterManager.java |  428 ++---
 .../java/org/apache/nifi/cluster/node/Node.java |    2 +-
 ...anagerProtocolServiceLocatorFactoryBean.java |    2 +-
 .../spring/WebClusterManagerFactoryBean.java    |    8 +-
 .../nifi-framework-core-api/pom.xml             |    4 +-
 .../apache/nifi/connectable/Connectable.java    |   75 +-
 .../controller/AbstractConfiguredComponent.java |   13 +-
 .../apache/nifi/controller/AbstractPort.java    |   32 +-
 .../nifi/controller/ConfiguredComponent.java    |    4 +-
 .../nifi/controller/ContentAvailability.java    |   24 +-
 .../nifi/controller/ProcessScheduler.java       |   58 +-
 .../apache/nifi/controller/ProcessorNode.java   |   18 +-
 .../nifi/controller/ReportingTaskNode.java      |   48 +-
 .../nifi/controller/StandardFlowFileQueue.java  |    6 +-
 .../apache/nifi/controller/StandardFunnel.java  |    7 +-
 .../controller/ValidationContextFactory.java    |    4 +-
 ...ControllerServiceAlreadyExistsException.java |    4 -
 ...ControllerServiceInstantiationException.java |   13 -
 .../reporting/ReportingTaskProvider.java        |   90 +-
 .../repository/FlowFileEventRepository.java     |   15 +-
 .../service/ControllerServiceNode.java          |   47 +-
 .../service/ControllerServiceProvider.java      |  168 +-
 .../service/ControllerServiceReference.java     |   17 +-
 .../service/ControllerServiceState.java         |   16 +-
 .../org/apache/nifi/groups/ProcessGroup.java    |  221 +--
 .../apache/nifi/groups/RemoteProcessGroup.java  |   56 +-
 .../RemoteProcessGroupPortDescriptor.java       |   42 +-
 .../org/apache/nifi/logging/LogRepository.java  |   18 +-
 .../nifi/nar/NarThreadContextClassLoader.java   |    5 +-
 .../org/apache/nifi/remote/RemoteGroupPort.java |    8 +-
 .../org/apache/nifi/remote/RootGroupPort.java   |   28 +-
 .../nifi/remote/protocol/ServerProtocol.java    |   62 +-
 .../nifi-framework/nifi-framework-core/pom.xml  |    1 +
 .../nifi/connectable/StandardConnection.java    |   15 +-
 .../nifi/controller/FileSystemSwapManager.java  |   49 +-
 .../apache/nifi/controller/FlowController.java  |  383 ++--
 .../nifi/controller/FlowFromDOMFactory.java     |   56 +-
 .../nifi/controller/FlowUnmarshaller.java       |   12 +-
 .../nifi/controller/StandardFlowSerializer.java |   65 +-
 .../nifi/controller/StandardFlowService.java    |   23 +-
 .../controller/StandardFlowSynchronizer.java    |  210 +--
 .../nifi/controller/StandardProcessorNode.java  |  110 +-
 .../apache/nifi/controller/StandardSnippet.java |   75 +-
 .../org/apache/nifi/controller/Template.java    |    2 +-
 .../apache/nifi/controller/TemplateManager.java |   48 +-
 .../reporting/AbstractReportingTaskNode.java    |   84 +-
 .../reporting/StandardReportingContext.java     |   12 +-
 .../StandardReportingInitializationContext.java |   12 +-
 .../repository/FileSystemRepository.java        |   35 +-
 .../controller/repository/ProcessContext.java   |   21 +-
 .../repository/ProvenanceEventEnricher.java     |    6 +-
 .../repository/RepositoryPurgeException.java    |   14 -
 .../repository/RingBufferEventRepository.java   |    2 +-
 .../repository/StandardFlowFileRecord.java      |    4 +-
 .../repository/StandardProcessSession.java      |  147 +-
 .../repository/StandardProvenanceReporter.java  |   15 +-
 .../repository/StandardRepositoryRecord.java    |   28 +-
 .../repository/VolatileContentRepository.java   |    6 +-
 .../repository/VolatileFlowFileRepository.java  |    2 +-
 .../WriteAheadFlowFileRepository.java           |   22 +-
 .../repository/claim/StandardContentClaim.java  |   11 +-
 .../claim/StandardContentClaimManager.java      |   10 -
 .../repository/io/ByteCountingOutputStream.java |    4 +-
 .../io/FlowFileAccessInputStream.java           |    6 +-
 .../scheduling/ConnectableProcessContext.java   |    8 +-
 .../scheduling/EventDrivenSchedulingAgent.java  |   38 +-
 .../scheduling/QuartzSchedulingAgent.java       |    2 +-
 .../controller/scheduling/ScheduleState.java    |    6 +-
 .../scheduling/StandardProcessScheduler.java    |  115 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  |   52 +-
 .../service/ControllerServiceLoader.java        |   53 +-
 ...dControllerServiceInitializationContext.java |    6 +-
 .../service/StandardControllerServiceNode.java  |   87 +-
 .../StandardControllerServiceProvider.java      |  304 ++--
 .../StandardControllerServiceReference.java     |    4 +-
 .../history/StandardMetricDescriptor.java       |   23 +-
 .../status/history/StandardStatusSnapshot.java  |    8 +
 .../VolatileComponentStatusRepository.java      |  441 ++---
 .../tasks/ContinuallyRunConnectableTask.java    |   11 +-
 .../tasks/ContinuallyRunProcessorTask.java      |   16 +-
 .../controller/tasks/ReportingTaskWrapper.java  |    6 +-
 .../apache/nifi/encrypt/StringEncryptor.java    |    2 +-
 .../java/org/apache/nifi/engine/FlowEngine.java |   26 +-
 .../nifi/events/VolatileBulletinRepository.java |    2 +-
 .../nifi/fingerprint/FingerprintFactory.java    |   88 +-
 .../nifi/groups/StandardProcessGroup.java       |   46 +-
 .../nifi/persistence/FlowConfigurationDAO.java  |    6 +-
 .../nifi/processor/SimpleProcessLogger.java     |    5 -
 .../nifi/processor/StandardProcessContext.java  |    6 +-
 .../nifi/processor/StandardPropertyValue.java   |    6 +-
 .../processor/StandardSchedulingContext.java    |    2 +-
 .../processor/StandardValidationContext.java    |   18 +-
 .../org/apache/nifi/remote/RemoteNiFiUtils.java |  120 +-
 .../nifi/remote/StandardRemoteProcessGroup.java |   48 +-
 .../org/apache/nifi/services/FlowService.java   |   11 +-
 .../nifi/spring/FlowControllerFactoryBean.java  |    2 +-
 .../spring/StandardFlowServiceFactoryBean.java  |    2 +-
 .../java/org/apache/nifi/util/DomUtils.java     |    6 +-
 .../org/apache/nifi/util/ReflectionUtils.java   |  156 +-
 .../java/org/apache/nifi/util/SnippetUtils.java |   32 +-
 .../src/main/resources/FlowConfiguration.xsd    |   38 +-
 .../nifi/cluster/HeartbeatPayloadTest.java      |    4 +-
 .../repository/TestStandardProcessSession.java  |  220 ++-
 .../StandardControllerServiceProviderTest.java  |   46 +-
 .../TestStandardControllerServiceProvider.java  |  104 +-
 .../controller/service/mock/DummyProcessor.java |   13 +-
 .../nifi/controller/service/mock/ServiceA.java  |   21 +-
 .../service/util/TestControllerService.java     |   58 +-
 .../processor/TestStandardPropertyValue.java    |    4 +-
 .../org/apache/nifi/nar/ExtensionManager.java   |   14 +-
 .../org/apache/nifi/nar/NarClassLoader.java     |   24 +-
 .../org/apache/nifi/nar/NarClassLoaders.java    |   40 +-
 .../java/org/apache/nifi/nar/NarUnpacker.java   |   14 +-
 .../java/org/apache/nifi/util/FileUtils.java    |   28 +-
 .../nifi-framework/nifi-runtime/pom.xml         |    2 +-
 .../java/org/apache/nifi/BootstrapListener.java |  501 +++---
 .../src/main/java/org/apache/nifi/NiFi.java     |   70 +-
 .../util/SslContextCreationException.java       |    1 +
 .../security/util/SslContextFactory.java        |   39 +-
 .../framework/security/util/SslException.java   |    1 +
 .../security/util/SslServerSocketFactory.java   |   19 +-
 ...SslServerSocketFactoryCreationException.java |    4 +-
 .../security/util/SslSocketFactory.java         |   17 +-
 .../util/SslSocketFactoryCreationException.java |    1 +
 .../security/util/SslContextFactoryTest.java    |   18 +-
 .../nifi-framework/nifi-site-to-site/pom.xml    |    4 +-
 .../util/RemoteProcessGroupUtils.java           |  120 +-
 .../nifi/remote/RemoteResourceFactory.java      |  114 +-
 .../nifi/remote/RemoteResourceManager.java      |   98 +-
 .../apache/nifi/remote/RemoteSiteListener.java  |    2 +-
 .../nifi/remote/SocketRemoteSiteListener.java   |  226 +--
 .../nifi/remote/StandardRemoteGroupPort.java    |  200 ++-
 .../nifi/remote/StandardRootGroupPort.java      |  243 +--
 .../exception/UnsupportedCodecException.java    |    5 +-
 .../socket/ClusterManagerServerProtocol.java    |   50 +-
 .../socket/SocketFlowFileServerProtocol.java    |  254 +--
 .../remote/TestStandardSiteToSiteProtocol.java  |   28 +-
 .../io/socket/TestSocketChannelStreams.java     |   62 +-
 .../io/socket/ssl/TestSSLSocketChannel.java     |  114 +-
 .../java/org/apache/nifi/action/Action.java     |    4 +-
 .../java/org/apache/nifi/action/Component.java  |    3 +-
 .../java/org/apache/nifi/action/Operation.java  |    1 +
 .../component/details/ComponentDetails.java     |    2 +-
 .../component/details/ExtensionDetails.java     |    2 +-
 .../details/RemoteProcessGroupDetails.java      |    2 +-
 .../nifi/action/details/ActionDetails.java      |    2 +-
 .../nifi/action/details/ConfigureDetails.java   |    2 +-
 .../nifi/action/details/ConnectDetails.java     |    2 +-
 .../apache/nifi/action/details/MoveDetails.java |    2 +-
 .../nifi/action/details/PurgeDetails.java       |   10 +-
 .../org/apache/nifi/web/server/JettyServer.java |  127 +-
 .../nifi-web/nifi-ui-extension/pom.xml          |   16 +-
 .../apache/nifi/ui/extension/UiExtension.java   |    8 +-
 .../nifi/ui/extension/UiExtensionMapping.java   |   17 +-
 .../nifi/audit/ControllerServiceAuditor.java    |   25 +-
 .../org/apache/nifi/audit/FunnelAuditor.java    |    4 +-
 .../java/org/apache/nifi/audit/NiFiAuditor.java |    2 +-
 .../java/org/apache/nifi/audit/PortAuditor.java |    4 +-
 .../apache/nifi/audit/ProcessGroupAuditor.java  |    2 +-
 .../org/apache/nifi/audit/ProcessorAuditor.java |    2 +-
 .../apache/nifi/audit/RelationshipAuditor.java  |    2 +-
 .../nifi/audit/RemoteProcessGroupAuditor.java   |    8 +-
 .../apache/nifi/audit/ReportingTaskAuditor.java |    3 +-
 .../org/apache/nifi/audit/SnippetAuditor.java   |    2 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  513 +++---
 .../nifi/web/StandardNiFiContentAccess.java     |   18 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |  192 +-
 .../StandardNiFiWebConfigurationContext.java    |  104 +-
 .../apache/nifi/web/StandardNiFiWebContext.java |    4 +-
 .../nifi/web/api/ApplicationResource.java       |    2 +-
 .../apache/nifi/web/api/ClusterResource.java    |   94 +-
 .../apache/nifi/web/api/ControllerResource.java |    8 +-
 .../nifi/web/api/ControllerServiceResource.java |  174 +-
 .../org/apache/nifi/web/api/FunnelResource.java |    3 +-
 .../apache/nifi/web/api/HistoryResource.java    |    6 +-
 .../apache/nifi/web/api/InputPortResource.java  |    3 +-
 .../org/apache/nifi/web/api/LabelResource.java  |    3 +-
 .../apache/nifi/web/api/OutputPortResource.java |    3 +-
 .../apache/nifi/web/api/ProcessorResource.java  |   20 +-
 .../web/api/RemoteProcessGroupResource.java     |    4 +-
 .../nifi/web/api/ReportingTaskResource.java     |  131 +-
 .../apache/nifi/web/api/TemplateResource.java   |    3 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |   90 +-
 .../ApplicationStartupContextListener.java      |   17 +-
 .../nifi/web/controller/ControllerFacade.java   |   47 +-
 .../org/apache/nifi/web/dao/ConnectionDAO.java  |   37 +-
 .../nifi/web/dao/ControllerServiceDAO.java      |   40 +-
 .../java/org/apache/nifi/web/dao/FunnelDAO.java |   25 +-
 .../java/org/apache/nifi/web/dao/LabelDAO.java  |   21 +-
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   29 +-
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   33 +-
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |   51 +-
 .../apache/nifi/web/dao/ReportingTaskDAO.java   |    2 +-
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   25 +-
 .../org/apache/nifi/web/dao/TemplateDAO.java    |    2 +-
 .../dao/impl/StandardControllerServiceDAO.java  |   49 +-
 .../nifi/web/dao/impl/StandardProcessorDAO.java |    2 +-
 .../dao/impl/StandardRemoteProcessGroupDAO.java |    1 -
 .../web/dao/impl/StandardReportingTaskDAO.java  |    4 +-
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |    8 +-
 .../ReportingTaskProviderFactoryBean.java       |    1 -
 .../org/apache/nifi/web/util/Availability.java  |    1 -
 .../org/apache/nifi/web/util/SnippetUtils.java  |  139 +-
 .../src/main/webapp/WEB-INF/web.xml             |    4 +
 .../nifi/integration/util/NiFiTestServer.java   |   58 +-
 .../java/org/apache/nifi/web/ContentAccess.java |    6 +-
 .../apache/nifi/web/ContentRequestContext.java  |   24 +-
 .../apache/nifi/web/DownloadableContent.java    |   12 +-
 .../nifi-web/nifi-web-content-viewer/pom.xml    |   14 +
 .../nifi/web/ContentViewerController.java       |   55 +-
 .../src/main/resources/META-INF/LICENSE         |  232 +++
 .../src/main/resources/META-INF/NOTICE          |   12 +-
 .../src/main/webapp/js/hexview/LICENSE          |   32 -
 .../main/webapp/js/hexview/hexview.default.css  |   34 +-
 .../src/main/webapp/js/hexview/hexview.js       |   33 +
 .../apache/nifi/web/ConfigurationRequest.java   |    9 +-
 .../apache/nifi/web/ConfigurationSnapshot.java  |    3 +-
 .../org/apache/nifi/web/FlowModification.java   |   16 +-
 .../nifi/web/OptimisticLockingManager.java      |   25 +-
 .../web/StandardOptimisticLockingManager.java   |   32 +-
 .../org/apache/nifi/web/UpdateRevision.java     |    6 +-
 .../org/apache/nifi/web/security/DnUtils.java   |   14 +-
 .../anonymous/NiFiAnonymousUserFilter.java      |    4 +-
 .../NiFiAuthenticationEntryPoint.java           |   11 +-
 .../authorization/NiFiAuthorizationService.java |   38 +-
 .../nifi/web/security/user/NiFiUserDetails.java |   16 +-
 .../nifi/web/security/user/NiFiUserUtils.java   |    6 +-
 .../x509/SubjectDnX509PrincipalExtractor.java   |    6 -
 .../security/x509/X509AuthenticationFilter.java |   50 +-
 .../security/x509/X509CertificateExtractor.java |    4 +-
 .../x509/ocsp/OcspCertificateValidator.java     |   43 +-
 .../NiFiAuthorizationServiceTest.java           |   91 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |   18 +
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |    1 +
 .../cluster-process-group-summary-dialog.jsp    |   36 +
 .../partials/summary/summary-content.jsp        |    3 +
 .../nifi-web-ui/src/main/webapp/css/summary.css |   83 +
 .../src/main/webapp/images/iconProcessGroup.png |  Bin 0 -> 1314 bytes
 .../webapp/js/nf/summary/nf-summary-table.js    |  448 ++++-
 .../src/main/resources/META-INF/NOTICE          |   68 +
 .../org/apache/nifi/processors/GeoEnrichIP.java |   51 +-
 .../nifi/processors/maxmind/DatabaseReader.java |   44 +-
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml  |   28 +-
 .../hadoop/AbstractHadoopProcessor.java         |   15 +-
 .../hadoop/CreateHadoopSequenceFile.java        |   28 +-
 .../nifi/processors/hadoop/FetchHDFS.java       |  141 +-
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  160 +-
 .../processors/hadoop/GetHDFSSequenceFile.java  |   18 +-
 .../nifi/processors/hadoop/KeyValueReader.java  |    6 +-
 .../apache/nifi/processors/hadoop/ListHDFS.java |  759 ++++----
 .../nifi/processors/hadoop/ValueReader.java     |    5 +-
 .../hadoop/util/ByteFilteringOutputStream.java  |   24 +-
 .../processors/hadoop/util/HDFSListing.java     |   94 +-
 .../hadoop/util/InputStreamWritable.java        |    6 +-
 .../nifi/processors/hadoop/util/LongSerDe.java  |   28 +-
 .../hadoop/util/OutputStreamWritable.java       |    3 +-
 .../hadoop/util/SequenceFileWriter.java         |   12 +-
 .../processors/hadoop/util/StringSerDe.java     |   24 +-
 .../src/main/resources/META-INF/NOTICE          |   29 +
 .../nifi-hl7-bundle/nifi-hl7-processors/pom.xml |  114 +-
 .../processors/hl7/ExtractHL7Attributes.java    |  365 ++--
 .../apache/nifi/processors/hl7/RouteHL7.java    |  300 ++--
 .../hl7/TestExtractHL7Attributes.java           |   30 +-
 .../apache/nifi/processors/kafka/GetKafka.java  |  438 ++---
 .../apache/nifi/processors/kafka/PutKafka.java  |  285 +--
 .../additionalDetails.html                      |   28 +-
 .../additionalDetails.html                      |   30 +-
 .../nifi/processors/kafka/TestGetKafka.java     |   58 +-
 .../nifi/processors/kafka/TestPutKafka.java     |  189 +-
 nifi/nifi-nar-bundles/nifi-kafka-bundle/pom.xml |   14 +-
 .../processors/kite/AbstractKiteProcessor.java  |   10 +-
 .../apache/nifi/processors/kite/AvroUtil.java   |   18 +-
 .../nifi/processors/kite/ConvertCSVToAvro.java  |  348 ++--
 .../nifi/processors/kite/ConvertJSONToAvro.java |    2 +-
 .../processors/kite/StoreInKiteDataset.java     |  208 +--
 .../processors/kite/TestCSVToAvroProcessor.java |  181 +-
 .../kite/TestConfigurationProperty.java         |   65 +-
 .../nifi/processors/kite/TestGetSchema.java     |  117 +-
 .../kite/TestJSONToAvroProcessor.java           |   58 +-
 .../kite/TestKiteProcessorsCluster.java         |  125 +-
 .../kite/TestKiteStorageProcessor.java          |  225 ++-
 .../apache/nifi/processors/kite/TestUtil.java   |  114 +-
 .../nifi-yandex-processors/pom.xml              |    4 +-
 .../nifi/processors/yandex/YandexTranslate.java |  381 ++--
 .../processors/yandex/model/Translation.java    |   55 +-
 .../nifi/processors/yandex/util/Languages.java  |  124 +-
 .../processors/yandex/TestYandexTranslate.java  |  317 ++--
 .../nifi-language-translation-bundle/pom.xml    |   28 +-
 .../nifi/provenance/IndexConfiguration.java     |   12 +-
 .../PersistentProvenanceRepository.java         |  648 +++----
 .../provenance/RepositoryConfiguration.java     |  100 +-
 .../nifi/provenance/StandardRecordReader.java   |  221 ++-
 .../nifi/provenance/StandardRecordWriter.java   |  110 +-
 .../provenance/expiration/ExpirationAction.java |    6 +-
 .../provenance/lucene/DeleteIndexAction.java    |   77 +-
 .../nifi/provenance/lucene/DocsReader.java      |   99 +-
 .../nifi/provenance/lucene/FieldNames.java      |    1 +
 .../nifi/provenance/lucene/IndexManager.java    |  467 +++++
 .../nifi/provenance/lucene/IndexSearch.java     |   67 +-
 .../nifi/provenance/lucene/IndexingAction.java  |  232 +--
 .../nifi/provenance/lucene/LineageQuery.java    |    6 +-
 .../nifi/provenance/lucene/LuceneUtil.java      |   26 +-
 .../provenance/rollover/CompressionAction.java  |   59 -
 .../provenance/rollover/RolloverAction.java     |   35 -
 .../provenance/serialization/RecordReader.java  |   72 +
 .../provenance/serialization/RecordReaders.java |  123 +-
 .../provenance/serialization/RecordWriter.java  |   25 +-
 .../provenance/serialization/RecordWriters.java |   13 +-
 .../nifi/provenance/toc/StandardTocReader.java  |  108 ++
 .../nifi/provenance/toc/StandardTocWriter.java  |  119 ++
 .../apache/nifi/provenance/toc/TocReader.java   |   62 +
 .../org/apache/nifi/provenance/toc/TocUtil.java |   40 +
 .../apache/nifi/provenance/toc/TocWriter.java   |   50 +
 .../TestPersistentProvenanceRepository.java     |  192 +-
 .../TestStandardRecordReaderWriter.java         |  189 ++
 .../org/apache/nifi/provenance/TestUtil.java    |   82 +
 .../provenance/toc/TestStandardTocReader.java   |   91 +
 .../provenance/toc/TestStandardTocWriter.java   |   42 +
 .../src/main/resources/META-INF/NOTICE          |   57 +
 .../nifi-twitter-processors/pom.xml             |   10 +-
 .../nifi/processors/twitter/GetTwitter.java     |  450 ++---
 .../web/StandardContentViewerController.java    |    9 +-
 .../main/webapp/META-INF/nifi-content-viewer    |   14 +
 .../nifi-standard-processors/pom.xml            |    3 +
 .../standard/AbstractJsonPathProcessor.java     |   16 +-
 .../standard/Base64EncodeContent.java           |   11 +-
 .../nifi/processors/standard/BinFiles.java      |  155 +-
 .../processors/standard/CompressContent.java    |   29 +-
 .../nifi/processors/standard/ControlRate.java   |   36 +-
 .../standard/ConvertCharacterSet.java           |   43 +-
 .../processors/standard/DetectDuplicate.java    |   28 +-
 .../processors/standard/DistributeLoad.java     |   76 +-
 .../nifi/processors/standard/EncodeContent.java |   27 +-
 .../processors/standard/EncryptContent.java     |   23 +-
 .../processors/standard/EvaluateJsonPath.java   |   99 +-
 .../standard/EvaluateRegularExpression.java     |   31 +-
 .../nifi/processors/standard/EvaluateXPath.java |   80 +-
 .../processors/standard/EvaluateXQuery.java     |   57 +-
 .../processors/standard/ExecuteProcess.java     |   10 +-
 .../standard/ExecuteStreamCommand.java          |   50 +-
 .../nifi/processors/standard/ExtractText.java   |   23 +-
 .../processors/standard/GenerateFlowFile.java   |    7 +-
 .../apache/nifi/processors/standard/GetFTP.java |   14 +-
 .../nifi/processors/standard/GetFile.java       |   26 +-
 .../processors/standard/GetFileTransfer.java    |    9 +-
 .../nifi/processors/standard/GetHTTP.java       |   69 +-
 .../nifi/processors/standard/GetJMSTopic.java   |   47 +-
 .../nifi/processors/standard/GetSFTP.java       |   16 +-
 .../processors/standard/HandleHttpRequest.java  |  463 ++---
 .../processors/standard/HandleHttpResponse.java |  126 +-
 .../nifi/processors/standard/HashAttribute.java |   64 +-
 .../nifi/processors/standard/HashContent.java   |   18 +-
 .../processors/standard/IdentifyMimeType.java   |   17 +-
 .../nifi/processors/standard/InvokeHTTP.java    |  127 +-
 .../nifi/processors/standard/JmsConsumer.java   |  104 +-
 .../nifi/processors/standard/ListenHTTP.java    |   18 +-
 .../nifi/processors/standard/ListenUDP.java     |  269 ++-
 .../nifi/processors/standard/LogAttribute.java  |    5 +-
 .../nifi/processors/standard/MergeContent.java  |  174 +-
 .../nifi/processors/standard/ModifyBytes.java   |    3 -
 .../processors/standard/MonitorActivity.java    |   27 +-
 .../nifi/processors/standard/PostHTTP.java      |  111 +-
 .../nifi/processors/standard/PutEmail.java      |  199 +--
 .../apache/nifi/processors/standard/PutFTP.java |   23 +-
 .../nifi/processors/standard/PutFile.java       |   29 +-
 .../processors/standard/PutFileTransfer.java    |   36 +-
 .../apache/nifi/processors/standard/PutJMS.java |   49 +-
 .../nifi/processors/standard/PutSFTP.java       |    3 +-
 .../nifi/processors/standard/ReplaceText.java   |   36 +-
 .../standard/ReplaceTextWithMapping.java        |   45 +-
 .../processors/standard/RouteOnAttribute.java   |   41 +-
 .../processors/standard/RouteOnContent.java     |   18 +-
 .../nifi/processors/standard/ScanAttribute.java |   15 +-
 .../nifi/processors/standard/ScanContent.java   |   21 +-
 .../processors/standard/SegmentContent.java     |   39 +-
 .../nifi/processors/standard/SplitContent.java  |   39 +-
 .../nifi/processors/standard/SplitJson.java     |   27 +-
 .../nifi/processors/standard/SplitText.java     |   50 +-
 .../nifi/processors/standard/SplitXml.java      |   18 +-
 .../nifi/processors/standard/TransformXml.java  |   20 +-
 .../nifi/processors/standard/UnpackContent.java |   54 +-
 .../nifi/processors/standard/ValidateXml.java   |   10 +-
 .../servlets/ContentAcknowledgmentServlet.java  |   11 +-
 .../standard/servlets/ListenHTTPServlet.java    |   29 +-
 .../nifi/processors/standard/util/Bin.java      |   38 +-
 .../processors/standard/util/BinManager.java    |   19 +-
 .../standard/util/DocumentReaderCallback.java   |   13 +-
 .../processors/standard/util/FTPTransfer.java   |    3 +-
 .../nifi/processors/standard/util/FTPUtils.java |   59 +-
 .../processors/standard/util/FileTransfer.java  |   29 +-
 .../processors/standard/util/JmsFactory.java    |    3 +-
 .../standard/util/JmsProcessingSummary.java     |  100 +-
 .../processors/standard/util/JmsProperties.java |    5 +-
 .../util/JsonPathExpressionValidator.java       |   86 +-
 .../standard/util/NLKBufferedReader.java        |   14 +-
 .../standard/util/SFTPConnection.java           |    9 -
 .../processors/standard/util/SFTPTransfer.java  |   26 +-
 .../processors/standard/util/SFTPUtils.java     |   12 -
 .../util/ValidatingBase32InputStream.java       |    4 +-
 .../util/ValidatingBase64InputStream.java       |    3 +-
 .../additionalDetails.html                      |    6 +-
 .../additionalDetails.html                      |    2 +-
 .../org/apache/tika/mime/custom-mimetypes.xml   |  144 +-
 .../processors/standard/CaptureServlet.java     |    9 +-
 .../processors/standard/HelloWorldServlet.java  |    3 +-
 .../standard/TestBase64EncodeContent.java       |    1 -
 .../standard/TestCompressContent.java           |   13 +-
 .../processors/standard/TestControlRate.java    |    3 +-
 .../standard/TestDetectDuplicate.java           |    9 +-
 .../processors/standard/TestDistributeLoad.java |    1 -
 .../processors/standard/TestEncryptContent.java |    1 -
 .../standard/TestEvaluateJsonPath.java          |    3 -
 .../processors/standard/TestEvaluateXPath.java  |    1 -
 .../processors/standard/TestExecuteProcess.java |   18 +-
 .../standard/TestExecuteStreamCommand.java      |    4 +-
 .../processors/standard/TestExtractText.java    |   23 +-
 .../nifi/processors/standard/TestGetHTTP.java   |   59 +-
 .../standard/TestHandleHttpRequest.java         |   30 +-
 .../standard/TestHandleHttpResponse.java        |   64 +-
 .../processors/standard/TestHashAttribute.java  |    2 +-
 .../processors/standard/TestHashContent.java    |    1 -
 .../standard/TestIdentifyMimeType.java          |    1 -
 .../processors/standard/TestInvokeHTTP.java     |   57 +-
 .../processors/standard/TestJmsConsumer.java    |  192 +-
 .../nifi/processors/standard/TestListenUDP.java |    1 -
 .../processors/standard/TestMergeContent.java   |   20 +-
 .../processors/standard/TestModifyBytes.java    |    1 -
 .../standard/TestMonitorActivity.java           |   18 +-
 .../nifi/processors/standard/TestPostHTTP.java  |   87 +-
 .../nifi/processors/standard/TestPutEmail.java  |   77 +-
 .../standard/TestReplaceTextLineByLine.java     |    1 -
 .../standard/TestReplaceTextWithMapping.java    |    4 +-
 .../standard/TestRouteOnAttribute.java          |    1 -
 .../processors/standard/TestRouteOnContent.java |    1 -
 .../processors/standard/TestScanAttribute.java  |    1 -
 .../processors/standard/TestSegmentContent.java |    1 -
 .../nifi/processors/standard/TestServer.java    |   65 +-
 .../processors/standard/TestSplitContent.java   |   21 +-
 .../nifi/processors/standard/TestSplitText.java |   10 -
 .../nifi/processors/standard/TestSplitXml.java  |    1 -
 .../processors/standard/TestTransformXml.java   |    5 -
 .../processors/standard/TestUnpackContent.java  |    2 -
 .../processors/standard/TestValidateXml.java    |    1 -
 .../standard/UserAgentTestingServlet.java       |    1 -
 .../test/resources/TestJson/json-sample.json    |  814 ++++-----
 .../ControllerStatusReportingTask.java          |   19 +-
 .../apache/nifi/controller/MonitorMemory.java   |    3 +-
 .../distributed/cache/client/Deserializer.java  |    8 +-
 .../cache/client/DistributedMapCacheClient.java |   69 +-
 .../cache/client/DistributedSetCacheClient.java |   27 +-
 .../distributed/cache/client/Serializer.java    |    9 +-
 .../distributed/cache/client/CommsSession.java  |   16 +-
 .../DistributedMapCacheClientService.java       |    7 +-
 .../DistributedSetCacheClientService.java       |    6 +-
 .../cache/client/SSLCommsSession.java           |   25 +-
 .../cache/client/StandardCommsSession.java      |    1 +
 .../additionalDetails.html                      |   60 +-
 .../cache/protocol/ProtocolHandshake.java       |   37 +-
 .../protocol/exception/HandshakeException.java  |    3 +-
 .../cache/server/AbstractCacheServer.java       |   25 +-
 .../distributed/cache/server/CacheRecord.java   |   12 +-
 .../distributed/cache/server/CacheServer.java   |    3 +-
 .../cache/server/DistributedCacheServer.java    |    3 +-
 .../cache/server/DistributedSetCacheServer.java |   13 +-
 .../cache/server/EvictionPolicy.java            |   24 +-
 .../cache/server/SetCacheServer.java            |   25 +-
 .../server/map/DistributedMapCacheServer.java   |   12 +-
 .../distributed/cache/server/map/MapCache.java  |    5 +
 .../cache/server/map/MapCacheRecord.java        |   19 +-
 .../cache/server/map/MapCacheServer.java        |   11 +-
 .../cache/server/map/MapPutResult.java          |    5 +-
 .../cache/server/map/PersistentMapCache.java    |   51 +-
 .../cache/server/map/SimpleMapCache.java        |   67 +-
 .../cache/server/set/PersistentSetCache.java    |   57 +-
 .../distributed/cache/server/set/SetCache.java  |    5 +-
 .../cache/server/set/SetCacheRecord.java        |   15 +-
 .../cache/server/set/SetCacheResult.java        |   11 +-
 .../cache/server/set/SimpleSetCache.java        |   41 +-
 .../additionalDetails.html                      |   62 +-
 .../cache/server/TestServerAndClient.java       |   15 +-
 .../nifi-http-context-map-api/pom.xml           |   34 +-
 .../org/apache/nifi/http/HttpContextMap.java    |   45 +-
 .../nifi-http-context-map/pom.xml               |   20 +-
 .../nifi/http/StandardHttpContextMap.java       |   83 +-
 .../index.html                                  |   36 +-
 .../nifi/ssl/StandardSSLContextService.java     |    3 +-
 .../apache/nifi/ssl/SSLContextServiceTest.java  |    4 +-
 .../apache/nifi/update/attributes/Criteria.java |   24 +-
 .../update/attributes/serde/CriteriaSerDe.java  |    8 +-
 .../processors/attributes/UpdateAttribute.java  |   59 +-
 .../update/attributes/api/RuleResource.java     |   10 +-
 nifi/pom.xml                                    |  222 +--
 1072 files changed, 26375 insertions(+), 25142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
index d8cb69a,b0e7f3c..9f751e4
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
@@@ -253,20 -253,20 +253,20 @@@ public final class SnippetUtils 
          }
          existingServiceDtos.add(serviceNodeDto);
  
-         for ( final Map.Entry<PropertyDescriptor, String> entry : serviceNode.getProperties().entrySet() ) {
+         for (final Map.Entry<PropertyDescriptor, String> entry : serviceNode.getProperties().entrySet()) {
              final PropertyDescriptor descriptor = entry.getKey();
              final String propertyValue = entry.getValue();
-             
-             if ( descriptor.getControllerServiceDefinition() != null && propertyValue != null ) {
+ 
 -            if (descriptor.getControllerServiceDefinition() != null) {
++            if (descriptor.getControllerServiceDefinition() != null && propertyValue != null) {
                  final ControllerServiceNode referencedNode = flowController.getControllerServiceNode(propertyValue);
-                 if ( referencedNode == null ) {
+                 if (referencedNode == null) {
                      throw new IllegalStateException("Controller Service with ID " + propertyValue + " is referenced in template but cannot be found");
                  }
-                 
+ 
                  final String referencedNodeId = referencedNode.getIdentifier();
-                 
+ 
                  final boolean alreadyPresent = isServicePresent(referencedNodeId, snippet.getControllerServices());
-                 if ( !alreadyPresent ) {
+                 if (!alreadyPresent) {
                      addControllerServicesToSnippet(snippet, referencedNode);
                  }
              }
@@@ -477,8 -476,8 +476,8 @@@
  
          // if there is any controller service that maps to another controller service, update the id's
          updateControllerServiceIdentifiers(snippetContentsCopy, serviceIdMap);
-         
-         // 
+ 
 -        // 
++        //
          // Copy ProcessGroups
          //
          // instantiate the process groups, renaming as necessary

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 44ebbf8,3294ead..91c21a0
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@@ -217,31 -216,4 +216,31 @@@ public abstract class AbstractHadoopPro
          };
      }
  
-     
++
 +    /**
 +     * Returns the relative path of the child that does not include the filename
 +     * or the root path.
-      * @param root
-      * @param child
-      * @return 
++     * @param root the path to relativize from
++     * @param child the path to relativize
++     * @return the relative path
 +     */
 +    public static String getPathDifference(final Path root, final Path child) {
 +        final int depthDiff = child.depth() - root.depth();
 +        if (depthDiff <= 1) {
 +            return "".intern();
 +        }
 +        String lastRoot = root.getName();
 +        Path childsParent = child.getParent();
 +        final StringBuilder builder = new StringBuilder();
 +        builder.append(childsParent.getName());
 +        for (int i = (depthDiff - 3); i >= 0; i--) {
 +            childsParent = childsParent.getParent();
 +            String name = childsParent.getName();
 +            if (name.equals(lastRoot) && childsParent.toString().endsWith(root.toString())) {
 +                break;
 +            }
 +            builder.insert(0, Path.SEPARATOR).insert(0, name);
 +        }
 +        return builder.toString();
 +    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index 06bb3c6,0000000..b5efce0
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@@ -1,126 -1,0 +1,125 @@@
 +/*
 + * 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.hadoop;
 +
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.net.URI;
 +import java.util.ArrayList;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.hadoop.fs.FSDataInputStream;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.security.AccessControlException;
 +import org.apache.nifi.annotation.behavior.SupportsBatching;
 +import org.apache.nifi.annotation.behavior.WritesAttribute;
 +import org.apache.nifi.annotation.documentation.CapabilityDescription;
 +import org.apache.nifi.annotation.documentation.SeeAlso;
 +import org.apache.nifi.annotation.documentation.Tags;
 +import org.apache.nifi.components.PropertyDescriptor;
 +import org.apache.nifi.flowfile.FlowFile;
 +import org.apache.nifi.processor.ProcessContext;
 +import org.apache.nifi.processor.ProcessSession;
 +import org.apache.nifi.processor.Relationship;
 +import org.apache.nifi.processor.exception.ProcessException;
 +import org.apache.nifi.processor.util.StandardValidators;
 +import org.apache.nifi.util.StopWatch;
 +
 +@SupportsBatching
 +@Tags({"hadoop", "hdfs", "get", "ingest", "fetch", "source"})
 +@CapabilityDescription("Retrieves a file from HDFS. The content of the incoming FlowFile is replaced by the content of the file in HDFS. "
- 		+ "The file in HDFS is left intact without any changes being made to it.")
++        + "The file in HDFS is left intact without any changes being made to it.")
 +@WritesAttribute(attribute="hdfs.failure.reason", description="When a FlowFile is routed to 'failure', this attribute is added indicating why the file could "
- 		+ "not be fetched from HDFS")
++        + "not be fetched from HDFS")
 +@SeeAlso({ListHDFS.class, GetHDFS.class, PutHDFS.class})
 +public class FetchHDFS extends AbstractHadoopProcessor {
++    static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()
++        .name("HDFS Filename")
++        .description("The name of the HDFS file to retrieve")
++        .required(true)
++        .expressionLanguageSupported(true)
++        .defaultValue("${path}/${filename}")
++        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
++        .build();
 +
- 	static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()
- 		.name("HDFS Filename")
- 		.description("The name of the HDFS file to retrieve")
- 		.required(true)
- 		.expressionLanguageSupported(true)
- 		.defaultValue("${path}/${filename}")
- 		.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- 		.build();
- 	
- 	static final Relationship REL_SUCCESS = new Relationship.Builder()
- 		.name("success")
- 		.description("FlowFiles will be routed to this relationship once they have been updated with the content of the HDFS file")
- 		.build();
- 	static final Relationship REL_FAILURE = new Relationship.Builder()
- 		.name("failure")
- 		.description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieved and trying again will likely not be helpful. "
- 				+ "This would occur, for instance, if the file is not found or if there is a permissions issue")
- 		.build();
- 	static final Relationship REL_COMMS_FAILURE = new Relationship.Builder()
- 		.name("comms.failure")
- 		.description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieve due to a communications failure. "
- 				+ "This generally indicates that the Fetch should be tried again.")
- 		.build();
++    static final Relationship REL_SUCCESS = new Relationship.Builder()
++        .name("success")
++        .description("FlowFiles will be routed to this relationship once they have been updated with the content of the HDFS file")
++        .build();
++    static final Relationship REL_FAILURE = new Relationship.Builder()
++        .name("failure")
++        .description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieved and trying again will likely not be helpful. "
++                + "This would occur, for instance, if the file is not found or if there is a permissions issue")
++        .build();
++    static final Relationship REL_COMMS_FAILURE = new Relationship.Builder()
++        .name("comms.failure")
++        .description("FlowFiles will be routed to this relationship if the content of the HDFS file cannot be retrieve due to a communications failure. "
++                + "This generally indicates that the Fetch should be tried again.")
++        .build();
 +
- 	@Override
- 	protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
- 		final List<PropertyDescriptor> properties = new ArrayList<>();
- 		properties.add(HADOOP_CONFIGURATION_RESOURCES);
- 		properties.add(FILENAME);
- 		return properties;
- 	}
- 	
- 	@Override
- 	public Set<Relationship> getRelationships() {
- 		final Set<Relationship> relationships = new HashSet<>();
- 		relationships.add(REL_SUCCESS);
- 		relationships.add(REL_FAILURE);
- 		relationships.add(REL_COMMS_FAILURE);
- 		return relationships;
- 	}
- 	
- 	@Override
- 	public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
- 		FlowFile flowFile = session.get();
- 		if ( flowFile == null ) {
- 			return;
- 		}
- 		
- 		final FileSystem hdfs = hdfsResources.get().getValue();
- 		final Path path = new Path(context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue());
- 		final URI uri = path.toUri();
- 		
- 		final StopWatch stopWatch = new StopWatch(true);
- 		try (final FSDataInputStream inStream = hdfs.open(path, 16384)) {
- 			flowFile = session.importFrom(inStream, flowFile);
- 			stopWatch.stop();
- 			getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
- 			session.getProvenanceReporter().modifyContent(flowFile, "Fetched content from " + uri, stopWatch.getDuration(TimeUnit.MILLISECONDS));
- 			session.transfer(flowFile, REL_SUCCESS);
- 		} catch (final FileNotFoundException | AccessControlException e) {
- 			getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
- 			flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
- 			flowFile = session.penalize(flowFile);
- 			session.transfer(flowFile, REL_FAILURE);
- 		} catch (final IOException e) {
- 			getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
- 			flowFile = session.penalize(flowFile);
- 			session.transfer(flowFile, REL_COMMS_FAILURE);
- 		}
- 	}
++    @Override
++    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
++        final List<PropertyDescriptor> properties = new ArrayList<>();
++        properties.add(HADOOP_CONFIGURATION_RESOURCES);
++        properties.add(FILENAME);
++        return properties;
++    }
++
++    @Override
++    public Set<Relationship> getRelationships() {
++        final Set<Relationship> relationships = new HashSet<>();
++        relationships.add(REL_SUCCESS);
++        relationships.add(REL_FAILURE);
++        relationships.add(REL_COMMS_FAILURE);
++        return relationships;
++    }
++
++    @Override
++    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
++        FlowFile flowFile = session.get();
++        if ( flowFile == null ) {
++            return;
++        }
++
++        final FileSystem hdfs = hdfsResources.get().getValue();
++        final Path path = new Path(context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue());
++        final URI uri = path.toUri();
++
++        final StopWatch stopWatch = new StopWatch(true);
++        try (final FSDataInputStream inStream = hdfs.open(path, 16384)) {
++            flowFile = session.importFrom(inStream, flowFile);
++            stopWatch.stop();
++            getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
++            session.getProvenanceReporter().modifyContent(flowFile, "Fetched content from " + uri, stopWatch.getDuration(TimeUnit.MILLISECONDS));
++            session.transfer(flowFile, REL_SUCCESS);
++        } catch (final FileNotFoundException | AccessControlException e) {
++            getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
++            flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
++            flowFile = session.penalize(flowFile);
++            session.transfer(flowFile, REL_FAILURE);
++        } catch (final IOException e) {
++            getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
++            flowFile = session.penalize(flowFile);
++            session.transfer(flowFile, REL_COMMS_FAILURE);
++        }
++    }
 +
 +}


[7/7] incubator-nifi git commit: Merge branch 'develop' into ListHDFS

Posted by ma...@apache.org.
Merge branch 'develop' into ListHDFS


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

Branch: refs/heads/ListHDFS
Commit: 00b686b0b2a66d4b238a1af0602edae6a36d7786
Parents: 3e767fb 8c00623
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Apr 28 10:13:11 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Apr 28 10:13:11 2015 -0400

----------------------------------------------------------------------
 .../client/socket/EndpointConnectionPool.java   |  2 +-
 .../nifi/processors/standard/GetHTTP.java       | 11 ++--
 .../DistributedMapCacheClientService.java       |  2 +
 .../DistributedSetCacheClientService.java       |  2 +
 .../additionalDetails.html                      | 45 -------------
 .../additionalDetails.html                      | 46 --------------
 .../index.html                                  | 67 --------------------
 .../additionalDetails.html                      | 49 --------------
 8 files changed, 11 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/00b686b0/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
----------------------------------------------------------------------


[4/7] incubator-nifi git commit: NIFI-533: Refactored for Unit Tests and added unit tests for ListHDFS

Posted by ma...@apache.org.
NIFI-533: Refactored for Unit Tests and added unit tests for ListHDFS


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

Branch: refs/heads/ListHDFS
Commit: e4f431561e6af7a603a5c8b7a82910f28dc6d600
Parents: dc7f7a8
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Apr 28 08:46:38 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Apr 28 08:46:38 2015 -0400

----------------------------------------------------------------------
 .../hadoop/AbstractHadoopProcessor.java         |  24 +-
 .../hadoop/CreateHadoopSequenceFile.java        |   2 +-
 .../nifi/processors/hadoop/FetchHDFS.java       |   2 +-
 .../apache/nifi/processors/hadoop/GetHDFS.java  |  10 +-
 .../processors/hadoop/GetHDFSSequenceFile.java  |  12 +-
 .../apache/nifi/processors/hadoop/ListHDFS.java |  42 ++-
 .../apache/nifi/processors/hadoop/PutHDFS.java  |  21 +-
 .../nifi/processors/hadoop/TestListHDFS.java    | 347 +++++++++++++++++++
 8 files changed, 416 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 91c21a0..355950f 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -70,7 +70,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
 
     // variables shared by all threads of this processor
     // Hadoop Configuration and FileSystem
-    protected final AtomicReference<Tuple<Configuration, FileSystem>> hdfsResources = new AtomicReference<>();
+    private final AtomicReference<Tuple<Configuration, FileSystem>> hdfsResources = new AtomicReference<>();
 
     @Override
     protected void init(ProcessorInitializationContext context) {
@@ -153,7 +153,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             String disableCacheName = String.format("fs.%s.impl.disable.cache", FileSystem.getDefaultUri(config).getScheme());
             config.set(disableCacheName, "true");
 
-            final FileSystem fs = FileSystem.get(config);
+            final FileSystem fs = getFileSystem(config);
             getLogger().info(
                     "Initialized a new HDFS File System with working dir: {} default block size: {} default replication: {} config: {}",
                     new Object[]{fs.getWorkingDirectory(), fs.getDefaultBlockSize(new Path(dir)),
@@ -165,6 +165,18 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
         }
     }
 
+    /**
+     * This exists in order to allow unit tests to override it so that they don't take several minutes waiting
+     * for UDP packets to be received
+     *
+     * @param config the configuration to use
+     * @return the FileSystem that is created for the given Configuration
+     * @throws IOException if unable to create the FileSystem
+     */
+    protected FileSystem getFileSystem(final Configuration config) throws IOException {
+        return FileSystem.get(config);
+    }
+
     /*
      * Drastically reduce the timeout of a socket connection from the default in FileSystem.get()
      */
@@ -243,4 +255,12 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
         }
         return builder.toString();
     }
+
+    protected Configuration getConfiguration() {
+        return hdfsResources.get().getKey();
+    }
+
+    protected FileSystem getFileSystem() {
+        return hdfsResources.get().getValue();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
index f462277..186a290 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
@@ -156,7 +156,7 @@ public class CreateHadoopSequenceFile extends AbstractHadoopProcessor {
         final String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key()) + ".sf";
         flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), fileName);
         try {
-            flowFile = sequenceFileWriter.writeSequenceFile(flowFile, session, hdfsResources.get().getKey(), compressionType);
+            flowFile = sequenceFileWriter.writeSequenceFile(flowFile, session, getConfiguration(), compressionType);
             session.transfer(flowFile, RELATIONSHIP_SUCCESS);
             getLogger().info("Transferred flowfile {} to {}", new Object[]{flowFile, RELATIONSHIP_SUCCESS});
         } catch (ProcessException e) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index b5efce0..4a52fb7 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -99,7 +99,7 @@ public class FetchHDFS extends AbstractHadoopProcessor {
             return;
         }
 
-        final FileSystem hdfs = hdfsResources.get().getValue();
+        final FileSystem hdfs = getFileSystem();
         final Path path = new Path(context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue());
         final URI uri = path.toUri();
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
index f7894d9..7aa534f 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
@@ -236,8 +236,8 @@ public class GetHDFS extends AbstractHadoopProcessor {
         abstractOnScheduled(context);
         // copy configuration values to pass them around cleanly
         processorConfig = new ProcessorConfiguration(context);
-        FileSystem fs = hdfsResources.get().getValue();
-        Path dir = new Path(context.getProperty(DIRECTORY).getValue());
+        final FileSystem fs = getFileSystem();
+        final Path dir = new Path(context.getProperty(DIRECTORY).getValue());
         if (!fs.exists(dir)) {
             throw new IOException("PropertyDescriptor " + DIRECTORY + " has invalid value " + dir + ". The directory does not exist.");
         }
@@ -330,8 +330,8 @@ public class GetHDFS extends AbstractHadoopProcessor {
     protected void processBatchOfFiles(final List<Path> files, final ProcessContext context, final ProcessSession session) {
         // process the batch of files
         FSDataInputStream stream = null;
-        Configuration conf = hdfsResources.get().getKey();
-        FileSystem hdfs = hdfsResources.get().getValue();
+        Configuration conf = getConfiguration();
+        FileSystem hdfs = getFileSystem();
         final boolean keepSourceFiles = context.getProperty(KEEP_SOURCE_FILE).asBoolean();
         final Double bufferSizeProp = context.getProperty(BUFFER_SIZE).asDataSize(DataUnit.B);
         int bufferSize = bufferSizeProp != null ? bufferSizeProp.intValue() : conf.getInt(BUFFER_SIZE_KEY,
@@ -398,7 +398,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
 
         if (System.currentTimeMillis() >= nextPollTime && listingLock.tryLock()) {
             try {
-                final FileSystem hdfs = hdfsResources.get().getValue();
+                final FileSystem hdfs = getFileSystem();
                 // get listing
                 listing = selectFiles(hdfs, processorConfig.getConfiguredRootDirPath(), null);
                 lastPollTime.set(System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java
index 22ba36b..f032ee4 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java
@@ -22,6 +22,9 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
@@ -34,10 +37,6 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processors.hadoop.util.SequenceFileReader;
 import org.apache.nifi.util.StopWatch;
-import org.apache.nifi.util.Tuple;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 
 /**
  * This processor is used to pull files from HDFS. The files being pulled in MUST be SequenceFile formatted files. The processor creates a flow file for each key/value entry in the ingested
@@ -80,9 +79,8 @@ public class GetHDFSSequenceFile extends GetHDFS {
 
     @Override
     protected void processBatchOfFiles(final List<Path> files, final ProcessContext context, final ProcessSession session) {
-        final Tuple<Configuration, FileSystem> hadoopResources = hdfsResources.get();
-        final Configuration conf = hadoopResources.getKey();
-        final FileSystem hdfs = hadoopResources.getValue();
+        final Configuration conf = getConfiguration();
+        final FileSystem hdfs = getFileSystem();
         final String flowFileContentValue = context.getProperty(FLOWFILE_CONTENT).getValue();
         final boolean keepSourceFiles = context.getProperty(KEEP_SOURCE_FILE).asBoolean();
         final Double bufferSizeProp = context.getProperty(BUFFER_SIZE).asDataSize(DataUnit.B);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
index 56a128a..151cbf2 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
@@ -118,12 +118,14 @@ public class ListHDFS extends AbstractHadoopProcessor {
     private volatile Long lastListingTime = null;
     private volatile Set<Path> latestPathsListed = new HashSet<>();
     private volatile boolean electedPrimaryNode = false;
-    private File persistenceFile = null;
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
         super.init(context);
-        persistenceFile = new File("conf/state/" + getIdentifier());
+    }
+
+    protected File getPersistenceFile() {
+        return new File("conf/state/" + getIdentifier());
     }
 
     @Override
@@ -143,7 +145,7 @@ public class ListHDFS extends AbstractHadoopProcessor {
         return relationships;
     }
 
-    private String getKey(final String directory) {
+    protected String getKey(final String directory) {
         return getIdentifier() + ".lastListingTime." + directory;
     }
 
@@ -169,18 +171,13 @@ public class ListHDFS extends AbstractHadoopProcessor {
     }
 
 
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
-        final String directory = context.getProperty(DIRECTORY).getValue();
-
+    private Long getMinTimestamp(final String directory, final DistributedMapCacheClient client) throws IOException {
         // Determine the timestamp for the last file that we've listed.
         Long minTimestamp = lastListingTime;
         if ( minTimestamp == null || electedPrimaryNode ) {
             // We haven't yet restored any state from local or distributed state - or it's been at least a minute since
             // we have performed a listing. In this case,
             // First, attempt to get timestamp from distributed cache service.
-            final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
-
             try {
                 final StringSerDe serde = new StringSerDe();
                 final String serializedState = client.get(getKey(directory), serde, serde);
@@ -197,14 +194,13 @@ public class ListHDFS extends AbstractHadoopProcessor {
                 this.lastListingTime = minTimestamp;
                 electedPrimaryNode = false; // no requirement to pull an update from the distributed cache anymore.
             } catch (final IOException ioe) {
-                getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished.");
-                context.yield();
-                return;
+                throw ioe;
             }
 
             // Check the persistence file. We want to use the latest timestamp that we have so that
             // we don't duplicate data.
             try {
+                final File persistenceFile = getPersistenceFile();
                 if ( persistenceFile.exists() ) {
                     try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
                         final Properties props = new Properties();
@@ -240,9 +236,25 @@ public class ListHDFS extends AbstractHadoopProcessor {
             }
         }
 
+        return minTimestamp;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final String directory = context.getProperty(DIRECTORY).getValue();
+        final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
+
+        final Long minTimestamp;
+        try {
+            minTimestamp = getMinTimestamp(directory, client);
+        } catch (final IOException ioe) {
+            getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished.");
+            context.yield();
+            return;
+        }
 
         // Pull in any file that is newer than the timestamp that we have.
-        final FileSystem hdfs = hdfsResources.get().getValue();
+        final FileSystem hdfs = getFileSystem();
         final boolean recursive = context.getProperty(RECURSE_SUBDIRS).asBoolean();
         final Path rootPath = new Path(directory);
 
@@ -311,7 +323,6 @@ public class ListHDFS extends AbstractHadoopProcessor {
                 }
 
                 // Attempt to save state to remote server.
-                final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
                 try {
                     client.put(getKey(directory), serializedState, new StringSerDe(), new StringSerDe());
                 } catch (final IOException ioe) {
@@ -397,11 +408,12 @@ public class ListHDFS extends AbstractHadoopProcessor {
         }
     }
 
-    private void persistLocalState(final String directory, final String serializedState) throws IOException {
+    protected void persistLocalState(final String directory, final String serializedState) throws IOException {
         // we need to keep track of all files that we pulled in that had a modification time equal to
         // lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
         // that have a mod time equal to that timestamp because more files may come in with the same timestamp
         // later in the same millisecond.
+        final File persistenceFile = getPersistenceFile();
         final File dir = persistenceFile.getParentFile();
         if ( !dir.exists() && !dir.mkdirs() ) {
             throw new IOException("Could not create directory " + dir.getAbsolutePath() + " in order to save local state");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
index 057f786..52cf475 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
@@ -32,10 +32,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -54,7 +54,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.stream.io.BufferedInputStream;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.StopWatch;
-import org.apache.nifi.util.Tuple;
 
 /**
  * This processor copies FlowFiles to HDFS.
@@ -183,8 +182,7 @@ public class PutHDFS extends AbstractHadoopProcessor {
         } else {
             dfsUmask = FsPermission.DEFAULT_UMASK;
         }
-        final Tuple<Configuration, FileSystem> resources = hdfsResources.get();
-        final Configuration conf = resources.getKey();
+        final Configuration conf = getConfiguration();
         FsPermission.setUMask(conf, new FsPermission(dfsUmask));
     }
 
@@ -195,26 +193,23 @@ public class PutHDFS extends AbstractHadoopProcessor {
             return;
         }
 
-        final Tuple<Configuration, FileSystem> resources = hdfsResources.get();
-        if (resources == null || resources.getKey() == null || resources.getValue() == null) {
+        final Configuration configuration = getConfiguration();
+        final FileSystem hdfs = getFileSystem();
+        if (configuration == null || hdfs == null) {
             getLogger().error("HDFS not configured properly");
             session.transfer(flowFile, REL_FAILURE);
             context.yield();
             return;
         }
-        final Configuration conf = resources.getKey();
-        final FileSystem hdfs = resources.getValue();
 
-        final Path configuredRootDirPath = new Path(context.getProperty(DIRECTORY).evaluateAttributeExpressions(flowFile)
-                .getValue());
+        final Path configuredRootDirPath = new Path(context.getProperty(DIRECTORY).evaluateAttributeExpressions(flowFile).getValue());
         final String conflictResponse = context.getProperty(CONFLICT_RESOLUTION).getValue();
 
         final Double blockSizeProp = context.getProperty(BLOCK_SIZE).asDataSize(DataUnit.B);
         final long blockSize = blockSizeProp != null ? blockSizeProp.longValue() : hdfs.getDefaultBlockSize(configuredRootDirPath);
 
         final Double bufferSizeProp = context.getProperty(BUFFER_SIZE).asDataSize(DataUnit.B);
-        final int bufferSize = bufferSizeProp != null ? bufferSizeProp.intValue() : conf.getInt(BUFFER_SIZE_KEY,
-                BUFFER_SIZE_DEFAULT);
+        final int bufferSize = bufferSizeProp != null ? bufferSizeProp.intValue() : configuration.getInt(BUFFER_SIZE_KEY, BUFFER_SIZE_DEFAULT);
 
         final Integer replicationProp = context.getProperty(REPLICATION_FACTOR).asInteger();
         final short replication = replicationProp != null ? replicationProp.shortValue() : hdfs
@@ -230,7 +225,7 @@ public class PutHDFS extends AbstractHadoopProcessor {
 
             // Create destination directory if it does not exist
             try {
-                if (!hdfs.getFileStatus(configuredRootDirPath).isDir()) {
+                if (!hdfs.getFileStatus(configuredRootDirPath).isDirectory()) {
                     throw new IOException(configuredRootDirPath.toString() + " already exists and is not a directory");
                 }
             } catch (FileNotFoundException fe) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e4f43156/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
new file mode 100644
index 0000000..499fd51
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/TestListHDFS.java
@@ -0,0 +1,347 @@
+/*
+ * 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.hadoop;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+import org.apache.nifi.annotation.notification.PrimaryNodeState;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestListHDFS {
+
+    private TestRunner runner;
+    private ListHDFSWithMockedFileSystem proc;
+    private MockCacheClient service;
+
+    @Before
+    public void setup() throws InitializationException {
+        proc = new ListHDFSWithMockedFileSystem();
+        runner = TestRunners.newTestRunner(proc);
+
+        service = new MockCacheClient();
+        runner.addControllerService("service", service);
+        runner.enableControllerService(service);
+
+        runner.setProperty(ListHDFS.HADOOP_CONFIGURATION_RESOURCES, "src/test/resources/core-site.xml");
+        runner.setProperty(ListHDFS.DIRECTORY, "/test");
+        runner.setProperty(ListHDFS.DISTRIBUTED_CACHE_SERVICE, "service");
+    }
+
+    @Test
+    public void testListingHasCorrectAttributes() {
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt")));
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 1);
+        final MockFlowFile mff = runner.getFlowFilesForRelationship(ListHDFS.REL_SUCCESS).get(0);
+        mff.assertAttributeEquals("path", "/test");
+        mff.assertAttributeEquals("filename", "testFile.txt");
+    }
+
+
+    @Test
+    public void testRecursive() {
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt")));
+
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, true, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testDir")));
+        proc.fileSystem.addFileStatus(new Path("/test/testDir"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testDir/1.txt")));
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 2);
+
+        final MockFlowFile mff1 = runner.getFlowFilesForRelationship(ListHDFS.REL_SUCCESS).get(0);
+        mff1.assertAttributeEquals("path", "/test");
+        mff1.assertAttributeEquals("filename", "testFile.txt");
+
+        final MockFlowFile mff2 = runner.getFlowFilesForRelationship(ListHDFS.REL_SUCCESS).get(1);
+        mff2.assertAttributeEquals("path", "/test/testDir");
+        mff2.assertAttributeEquals("filename", "1.txt");
+    }
+
+    @Test
+    public void testNotRecursive() {
+        runner.setProperty(ListHDFS.RECURSE_SUBDIRS, "false");
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt")));
+
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, true, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testDir")));
+        proc.fileSystem.addFileStatus(new Path("/test/testDir"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testDir/1.txt")));
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 1);
+
+        final MockFlowFile mff1 = runner.getFlowFilesForRelationship(ListHDFS.REL_SUCCESS).get(0);
+        mff1.assertAttributeEquals("path", "/test");
+        mff1.assertAttributeEquals("filename", "testFile.txt");
+    }
+
+
+    @Test
+    public void testNoListUntilUpdateFromRemoteOnPrimaryNodeChange() {
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile.txt")));
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 1);
+
+        final MockFlowFile mff1 = runner.getFlowFilesForRelationship(ListHDFS.REL_SUCCESS).get(0);
+        mff1.assertAttributeEquals("path", "/test");
+        mff1.assertAttributeEquals("filename", "testFile.txt");
+
+        runner.clearTransferState();
+
+        // add new file to pull
+        proc.fileSystem.addFileStatus(new Path("/test"), new FileStatus(1L, false, 1, 1L, 0L, 0L, create777(), "owner", "group", new Path("/test/testFile2.txt")));
+
+        // trigger primary node change
+        proc.onPrimaryNodeChange(PrimaryNodeState.ELECTED_PRIMARY_NODE);
+
+        // cause calls to service to fail
+        service.failOnCalls = true;
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0);
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 0);
+
+        final String key = proc.getKey("/test");
+
+        // wait just to a bit to ensure that the timestamp changes when we update the service
+        final Object curVal = service.values.get(key);
+        try {
+            Thread.sleep(10L);
+        } catch (final InterruptedException ie) {
+        }
+
+        service.failOnCalls = false;
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ListHDFS.REL_SUCCESS, 1);
+
+        // ensure state saved both locally & remotely
+        assertTrue(proc.localStateSaved);
+        assertNotNull(service.values.get(key));
+        assertNotSame(curVal, service.values.get(key));
+    }
+
+
+    private FsPermission create777() {
+        return new FsPermission((short) 0777);
+    }
+
+
+    private class ListHDFSWithMockedFileSystem extends ListHDFS {
+        private final MockFileSystem fileSystem = new MockFileSystem();
+        private boolean localStateSaved = false;
+
+        @Override
+        protected FileSystem getFileSystem() {
+            return fileSystem;
+        }
+
+        @Override
+        protected File getPersistenceFile() {
+            return new File("target/conf/state-file");
+        }
+
+        @Override
+        protected FileSystem getFileSystem(Configuration config) throws IOException {
+            return fileSystem;
+        }
+
+        @Override
+        protected void persistLocalState(String directory, String serializedState) throws IOException {
+            super.persistLocalState(directory, serializedState);
+            localStateSaved = true;
+        }
+    }
+
+
+    private class MockFileSystem extends FileSystem {
+        private final Map<Path, Set<FileStatus>> fileStatuses = new HashMap<>();
+
+        public void addFileStatus(final Path parent, final FileStatus child) {
+            Set<FileStatus> children = fileStatuses.get(parent);
+            if ( children == null ) {
+                children = new HashSet<>();
+                fileStatuses.put(parent, children);
+            }
+
+            children.add(child);
+        }
+
+
+        @Override
+        public long getDefaultBlockSize() {
+            return 1024L;
+        }
+
+        @Override
+        public short getDefaultReplication() {
+            return 1;
+        }
+
+        @Override
+        public URI getUri() {
+            return null;
+        }
+
+        @Override
+        public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+            return null;
+        }
+
+        @Override
+        public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
+            return null;
+        }
+
+        @Override
+        public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException {
+            return null;
+        }
+
+        @Override
+        public boolean rename(Path src, Path dst) throws IOException {
+            return false;
+        }
+
+        @Override
+        public boolean delete(Path f, boolean recursive) throws IOException {
+            return false;
+        }
+
+        @Override
+        public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
+            final Set<FileStatus> statuses = fileStatuses.get(f);
+            if ( statuses == null ) {
+                return new FileStatus[0];
+            }
+
+            return statuses.toArray(new FileStatus[statuses.size()]);
+        }
+
+        @Override
+        public void setWorkingDirectory(Path new_dir) {
+
+        }
+
+        @Override
+        public Path getWorkingDirectory() {
+            return new Path(new File(".").getAbsolutePath());
+        }
+
+        @Override
+        public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+            return false;
+        }
+
+        @Override
+        public FileStatus getFileStatus(Path f) throws IOException {
+            return null;
+        }
+
+    }
+
+
+    private class MockCacheClient extends AbstractControllerService implements DistributedMapCacheClient {
+        private ConcurrentMap<Object, Object> values = new ConcurrentHashMap<>();
+        private boolean failOnCalls = false;
+
+        private void verifyNotFail() throws IOException {
+            if ( failOnCalls ) {
+                throw new IOException("Could not call to remote service because Unit Test marked service unavailable");
+            }
+        }
+
+        @Override
+        public <K, V> boolean putIfAbsent(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            final Object retValue = values.putIfAbsent(key, value);
+            return (retValue == null);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V getAndPutIfAbsent(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer, Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.putIfAbsent(key, value);
+        }
+
+        @Override
+        public <K> boolean containsKey(K key, Serializer<K> keySerializer) throws IOException {
+            verifyNotFail();
+            return values.containsKey(key);
+        }
+
+        @Override
+        public <K, V> void put(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            values.put(key, value);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V get(K key, Serializer<K> keySerializer, Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.get(key);
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public <K> boolean remove(K key, Serializer<K> serializer) throws IOException {
+            verifyNotFail();
+            values.remove(key);
+            return true;
+        }
+    }
+}


[5/7] incubator-nifi git commit: Merge branch 'develop' into ListHDFS

Posted by ma...@apache.org.
Merge branch 'develop' into ListHDFS


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

Branch: refs/heads/ListHDFS
Commit: 3e767fbdf3af66d3348d974a1c8a669819d5f6d9
Parents: e4f4315 6c3256e
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Apr 28 08:47:23 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Apr 28 08:47:23 2015 -0400

----------------------------------------------------------------------
 nifi-nar-maven-plugin/pom.xml                   |   7 +
 nifi-parent/pom.xml                             |  67 +++--
 .../flowfile/attributes/CoreAttributes.java     |  12 +-
 .../apache/nifi/remote/VersionNegotiator.java   |  15 +-
 .../TransmissionDisabledException.java          |   3 +-
 .../nifi/remote/io/CompressionOutputStream.java |   3 +-
 .../remote/io/socket/BufferStateManager.java    |   4 +-
 .../socket/ssl/SSLSocketChannelInputStream.java |   3 +-
 .../ssl/SSLSocketChannelOutputStream.java       |   3 +-
 .../nifi/stream/io/BufferedInputStream.java     |   7 +-
 .../nifi/stream/io/BufferedOutputStream.java    |  31 +--
 .../nifi/stream/io/ByteArrayInputStream.java    | 113 +++-----
 .../nifi/stream/io/ByteArrayOutputStream.java   |  80 ++----
 .../stream/io/ByteCountingOutputStream.java     |   5 +-
 .../apache/nifi/stream/io/DataOutputStream.java | 113 +++-----
 .../apache/nifi/stream/io/GZIPOutputStream.java |   4 +-
 .../stream/io/LeakyBucketStreamThrottler.java   |   3 +-
 .../stream/io/MinimumLengthInputStream.java     |   3 +-
 .../nifi/stream/io/NonCloseableInputStream.java |   5 +-
 .../org/apache/nifi/stream/io/StreamUtils.java  |  30 +--
 .../apache/nifi/stream/io/ZipOutputStream.java  |   5 +-
 .../java/org/apache/nifi/util/EscapeUtils.java  |   4 +-
 .../java/org/apache/nifi/util/LongHolder.java   |   6 +-
 .../apache/nifi/util/NaiveSearchRingBuffer.java |  21 +-
 .../java/org/apache/nifi/util/RingBuffer.java   |  26 +-
 .../java/org/apache/nifi/util/StopWatch.java    |   3 +-
 .../org/apache/nifi/util/file/FileUtils.java    | 114 +++-----
 .../file/monitor/CompoundUpdateMonitor.java     |   7 +-
 .../file/monitor/SynchronousFileWatcher.java    |   6 +-
 .../org/apache/nifi/util/search/Search.java     |  18 +-
 .../org/apache/nifi/util/search/SearchTerm.java |   5 +-
 .../nifi/web/api/dto/BulletinBoardDTO.java      |   3 +-
 .../apache/nifi/web/api/dto/BulletinDTO.java    |   7 +-
 .../nifi/web/api/dto/BulletinQueryDTO.java      |   3 +-
 .../apache/nifi/web/api/dto/ConnectableDTO.java |   6 +-
 .../apache/nifi/web/api/dto/ConnectionDTO.java  |  25 +-
 .../web/api/dto/ControllerConfigurationDTO.java |   6 +-
 .../apache/nifi/web/api/dto/ControllerDTO.java  |  17 +-
 .../nifi/web/api/dto/ControllerServiceDTO.java  |  10 +-
 ...ontrollerServiceReferencingComponentDTO.java |  19 +-
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   3 +-
 .../org/apache/nifi/web/api/dto/LabelDTO.java   |   1 -
 .../nifi/web/api/dto/NiFiComponentDTO.java      |   3 +-
 .../org/apache/nifi/web/api/dto/PortDTO.java    |  10 +-
 .../nifi/web/api/dto/ProcessGroupDTO.java       |   3 +-
 .../nifi/web/api/dto/ProcessorConfigDTO.java    |  32 +--
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  10 +-
 .../nifi/web/api/dto/PropertyDescriptorDTO.java |  22 +-
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  15 +-
 .../web/api/dto/RemoteProcessGroupPortDTO.java  |   3 +-
 .../nifi/web/api/dto/ReportingTaskDTO.java      |  19 +-
 .../apache/nifi/web/api/dto/RevisionDTO.java    |  11 +-
 .../org/apache/nifi/web/api/dto/SnippetDTO.java |  48 ++--
 .../org/apache/nifi/web/api/dto/UserDTO.java    |   3 +-
 .../web/api/dto/provenance/ProvenanceDTO.java   |   3 +-
 .../api/dto/provenance/ProvenanceEventDTO.java  |  35 +--
 .../provenance/lineage/LineageRequestDTO.java   |   7 +-
 .../provenance/lineage/ProvenanceNodeDTO.java   |   3 +-
 .../web/api/dto/status/ControllerStatusDTO.java |   3 +-
 .../nifi/web/api/dto/status/PortStatusDTO.java  |   6 +-
 .../api/dto/status/ProcessGroupStatusDTO.java   |   9 +-
 .../web/api/dto/status/ProcessorStatusDTO.java  |   9 +-
 .../dto/status/RemoteProcessGroupStatusDTO.java |   6 +-
 .../apache/nifi/web/api/entity/AboutEntity.java |   4 +-
 .../nifi/web/api/entity/ActionEntity.java       |   4 +-
 .../nifi/web/api/entity/AuthorityEntity.java    |   4 +-
 .../nifi/web/api/entity/BannerEntity.java       |   4 +-
 .../web/api/entity/BulletinBoardEntity.java     |   4 +-
 .../entity/ClusterConnectionStatusEntity.java   |   4 +-
 .../nifi/web/api/entity/ClusterEntity.java      |   4 +-
 .../web/api/entity/ClusterPortStatusEntity.java |   4 +-
 .../entity/ClusterProcessGroupStatusEntity.java |   4 +-
 .../entity/ClusterProcessorStatusEntity.java    |   4 +-
 .../ClusterRemoteProcessGroupStatusEntity.java  |   5 +-
 .../api/entity/ClusterSearchResultsEntity.java  |   4 +-
 .../web/api/entity/ClusterStatusEntity.java     |   4 +-
 .../api/entity/ClusterStatusHistoryEntity.java  |   4 +-
 .../web/api/entity/ComponentHistoryEntity.java  |   4 +-
 .../nifi/web/api/entity/ConnectionEntity.java   |   4 +-
 .../nifi/web/api/entity/ConnectionsEntity.java  |   4 +-
 .../entity/ControllerConfigurationEntity.java   |   4 +-
 .../nifi/web/api/entity/ControllerEntity.java   |   4 +-
 .../web/api/entity/ControllerServiceEntity.java |   4 +-
 ...ollerServiceReferencingComponentsEntity.java |   8 +-
 .../entity/ControllerServiceTypesEntity.java    |   4 +-
 .../api/entity/ControllerServicesEntity.java    |   4 +-
 .../web/api/entity/ControllerStatusEntity.java  |   4 +-
 .../nifi/web/api/entity/CounterEntity.java      |   4 +-
 .../nifi/web/api/entity/CountersEntity.java     |   7 +-
 .../nifi/web/api/entity/FlowSnippetEntity.java  |   4 +-
 .../nifi/web/api/entity/FunnelEntity.java       |   4 +-
 .../nifi/web/api/entity/FunnelsEntity.java      |   4 +-
 .../nifi/web/api/entity/HistoryEntity.java      |   4 +-
 .../nifi/web/api/entity/InputPortEntity.java    |   4 +-
 .../nifi/web/api/entity/InputPortsEntity.java   |   4 +-
 .../apache/nifi/web/api/entity/LabelEntity.java |   4 +-
 .../nifi/web/api/entity/LabelsEntity.java       |   4 +-
 .../nifi/web/api/entity/LineageEntity.java      |   4 +-
 .../apache/nifi/web/api/entity/NodeEntity.java  |   4 +-
 .../nifi/web/api/entity/NodeStatusEntity.java   |   4 +-
 .../api/entity/NodeSystemDiagnosticsEntity.java |   4 +-
 .../nifi/web/api/entity/OutputPortEntity.java   |   4 +-
 .../nifi/web/api/entity/OutputPortsEntity.java  |   4 +-
 .../web/api/entity/PrioritizerTypesEntity.java  |   4 +-
 .../nifi/web/api/entity/ProcessGroupEntity.java |   4 +-
 .../api/entity/ProcessGroupStatusEntity.java    |   4 +-
 .../web/api/entity/ProcessGroupsEntity.java     |   4 +-
 .../nifi/web/api/entity/ProcessorEntity.java    |   4 +-
 .../web/api/entity/ProcessorTypesEntity.java    |   4 +-
 .../nifi/web/api/entity/ProcessorsEntity.java   |   4 +-
 .../api/entity/PropertyDescriptorEntity.java    |   4 +-
 .../web/api/entity/ProvenanceEventEntity.java   |   4 +-
 .../web/api/entity/ProvenanceOptionsEntity.java |   4 +-
 .../api/entity/RemoteProcessGroupEntity.java    |   4 +-
 .../entity/RemoteProcessGroupPortEntity.java    |   4 +-
 .../api/entity/RemoteProcessGroupsEntity.java   |   4 +-
 .../web/api/entity/ReportingTaskEntity.java     |   4 +-
 .../api/entity/ReportingTaskTypesEntity.java    |   4 +-
 .../web/api/entity/ReportingTasksEntity.java    |   4 +-
 .../web/api/entity/SearchResultsEntity.java     |   6 +-
 .../nifi/web/api/entity/SnippetEntity.java      |   4 +-
 .../web/api/entity/StatusHistoryEntity.java     |   4 +-
 .../web/api/entity/SystemDiagnosticsEntity.java |   4 +-
 .../nifi/web/api/entity/TemplateEntity.java     |   4 +-
 .../nifi/web/api/entity/TemplatesEntity.java    |   4 +-
 .../apache/nifi/web/api/entity/UserEntity.java  |   4 +-
 .../nifi/web/api/entity/UserGroupEntity.java    |   4 +-
 .../web/api/entity/UserSearchResultsEntity.java |   5 +-
 .../apache/nifi/web/api/entity/UsersEntity.java |   4 +-
 .../nifi-file-authorization-provider/pom.xml    |   8 +
 .../FileAuthorizationProvider.java              |  90 +------
 .../FileAuthorizationProviderTest.java          |  63 ++---
 .../org/apache/nifi/cluster/event/Event.java    |   6 +-
 .../apache/nifi/cluster/event/EventManager.java |   8 +-
 .../cluster/event/impl/EventManagerImpl.java    |   6 +-
 .../cluster/firewall/ClusterNodeFirewall.java   |   6 +-
 .../impl/FileBasedClusterNodeFirewall.java      |  16 +-
 .../apache/nifi/cluster/flow/DataFlowDao.java   |   3 +-
 .../cluster/flow/DataFlowManagementService.java |  23 +-
 .../nifi/cluster/flow/StaleFlowException.java   |   3 +-
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java |  30 +--
 .../impl/DataFlowManagementServiceImpl.java     |  19 +-
 .../nifi/cluster/manager/ClusterManager.java    | 100 +++----
 .../cluster/manager/HttpClusterManager.java     | 111 +++-----
 .../cluster/manager/HttpRequestReplicator.java  |  41 +--
 .../cluster/manager/HttpResponseMapper.java     |   3 +-
 .../nifi/cluster/manager/NodeResponse.java      |  57 ++--
 .../ConnectingNodeMutableRequestException.java  |   3 +-
 ...DisconnectedNodeMutableRequestException.java |   3 +-
 .../exception/IllegalClusterStateException.java |   3 +-
 .../exception/IllegalNodeDeletionException.java |   3 +-
 .../IllegalNodeDisconnectionException.java      |   4 +-
 .../IllegalNodeReconnectionException.java       |   3 +-
 .../IneligiblePrimaryNodeException.java         |   3 +-
 .../exception/MutableRequestException.java      |   5 +-
 .../exception/NoConnectedNodesException.java    |   3 +-
 .../exception/NoResponseFromNodesException.java |   5 +-
 .../exception/NodeDisconnectionException.java   |   3 +-
 .../PrimaryRoleAssignmentException.java         |   3 +-
 .../SafeModeMutableRequestException.java        |   3 +-
 .../manager/exception/UnknownNodeException.java |   3 +-
 .../exception/UriConstructionException.java     |   4 +-
 .../manager/impl/HttpRequestReplicatorImpl.java |  42 +--
 .../manager/impl/HttpResponseMapperImpl.java    |   9 +-
 .../cluster/manager/impl/WebClusterManager.java | 261 +++++++------------
 .../java/org/apache/nifi/cluster/node/Node.java |  40 +--
 ...anagerProtocolServiceLocatorFactoryBean.java |   8 +-
 .../spring/WebClusterManagerFactoryBean.java    |   4 +-
 .../event/impl/EventManagerImplTest.java        |   5 +-
 .../impl/FileBasedClusterNodeFirewallTest.java  |   4 +-
 .../impl/HttpRequestReplicatorImplTest.java     |   5 +-
 .../impl/HttpResponseMapperImplTest.java        |   6 +-
 .../cluster/manager/testutils/HttpRequest.java  |   7 +-
 .../cluster/manager/testutils/HttpResponse.java |   3 +-
 .../manager/testutils/HttpResponseAction.java   |   4 +-
 .../cluster/manager/testutils/HttpServer.java   |   3 +-
 .../ClusterManagerProtocolSenderImplTest.java   |   6 +-
 .../impl/ClusterServiceLocatorTest.java         |   6 +-
 .../impl/ClusterServicesBroadcasterTest.java    |   3 +-
 .../impl/MulticastProtocolListenerTest.java     |   2 +-
 .../impl/NodeProtocolSenderImplTest.java        |   3 +-
 .../impl/SocketProtocolListenerTest.java        |   3 +-
 .../apache/nifi/cluster/HeartbeatPayload.java   |   3 +-
 .../org/apache/nifi/connectable/LocalPort.java  |   5 +-
 .../nifi/connectable/StandardConnection.java    |  10 +-
 .../nifi/controller/FileSystemSwapManager.java  |   6 +-
 .../apache/nifi/controller/FlowController.java  | 214 +++++----------
 .../controller/FlowSerializationException.java  |   3 +-
 .../apache/nifi/controller/FlowSerializer.java  |   3 +-
 .../FlowSynchronizationException.java           |   3 +-
 .../nifi/controller/FlowSynchronizer.java       |  25 +-
 .../nifi/controller/FlowUnmarshaller.java       |   4 +-
 .../controller/StandardFlowSynchronizer.java    |  12 +-
 .../nifi/controller/StandardProcessorNode.java  |  64 ++---
 .../apache/nifi/controller/TemplateManager.java |  19 +-
 .../controller/UninheritableFlowException.java  |   3 +-
 .../repository/FileSystemRepository.java        |   4 +-
 .../controller/repository/ProcessContext.java   |  18 +-
 .../repository/ProvenanceEventEnricher.java     |   3 +-
 .../repository/RepositoryPurgeException.java    |   3 +-
 .../repository/StandardFlowFileRecord.java      |   7 +-
 .../repository/StandardProcessSession.java      |  34 +--
 .../repository/StandardProvenanceReporter.java  |   6 +-
 .../repository/StandardRepositoryRecord.java    |   3 +-
 .../StandardRepositoryStatusReport.java         |  12 +-
 .../repository/VolatileContentRepository.java   |  30 +--
 .../repository/VolatileFlowFileRepository.java  |   4 +-
 .../WriteAheadFlowFileRepository.java           |  25 +-
 .../repository/claim/ContentDirection.java      |   9 +-
 .../repository/claim/StandardContentClaim.java  |   7 +-
 .../io/DisableOnCloseInputStream.java           |   4 +-
 .../io/DisableOnCloseOutputStream.java          |   4 +-
 .../io/FlowFileAccessInputStream.java           |  11 +-
 .../io/FlowFileAccessOutputStream.java          |   9 +-
 .../controller/repository/io/LongHolder.java    |   6 +-
 .../scheduling/ConnectableProcessContext.java   |   3 +-
 .../controller/scheduling/ScheduleState.java    |  12 +-
 .../scheduling/StandardProcessScheduler.java    |  18 +-
 .../StandardControllerServiceProvider.java      |   6 +-
 .../status/history/StandardStatusSnapshot.java  |   2 +-
 .../tasks/ContinuallyRunConnectableTask.java    |   4 +-
 .../tasks/ContinuallyRunProcessorTask.java      |   4 +-
 .../nifi/controller/tasks/ExpireFlowFiles.java  |   4 +-
 .../apache/nifi/encrypt/StringEncryptor.java    |  12 +-
 .../java/org/apache/nifi/engine/FlowEngine.java |  16 +-
 .../nifi/events/VolatileBulletinRepository.java |   5 +-
 .../nifi/fingerprint/FingerprintFactory.java    |  17 +-
 .../nifi/groups/StandardProcessGroup.java       |  13 +-
 .../org/apache/nifi/lifecycle/LifeCycle.java    |  17 +-
 .../nifi/lifecycle/LifeCycleException.java      |   3 +-
 .../nifi/lifecycle/LifeCycleStartException.java |   4 +-
 .../nifi/lifecycle/LifeCycleStopException.java  |   3 +-
 .../nifi/persistence/FlowConfigurationDAO.java  |  33 +--
 .../nifi/processor/StandardProcessContext.java  |   3 +-
 .../nifi/processor/StandardPropertyValue.java   |  12 +-
 .../org/apache/nifi/remote/RemoteNiFiUtils.java |  11 +-
 .../nifi/remote/StandardRemoteProcessGroup.java |  49 ++--
 .../org/apache/nifi/services/FlowService.java   |  55 ++--
 .../nifi/spring/FlowControllerFactoryBean.java  |   4 +-
 .../spring/StandardFlowServiceFactoryBean.java  |   4 +-
 .../apache/nifi/util/ComponentStatusReport.java |   4 +-
 .../org/apache/nifi/util/ReflectionUtils.java   |  71 ++---
 .../java/org/apache/nifi/util/SnippetUtils.java |   6 +-
 .../apache/nifi/audit/ControllerAuditor.java    |  38 ++-
 .../nifi/audit/ControllerServiceAuditor.java    |  93 ++++---
 .../org/apache/nifi/audit/FunnelAuditor.java    |  33 ++-
 .../java/org/apache/nifi/audit/NiFiAuditor.java |   8 +-
 .../java/org/apache/nifi/audit/PortAuditor.java |  45 ++--
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  42 ++-
 .../org/apache/nifi/audit/ProcessorAuditor.java |  66 ++---
 .../apache/nifi/audit/RelationshipAuditor.java  |  72 +++--
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  56 ++--
 .../apache/nifi/audit/ReportingTaskAuditor.java |  64 +++--
 .../org/apache/nifi/audit/SnippetAuditor.java   |  40 ++-
 .../IllegalClusterResourceRequestException.java |   3 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |  36 +--
 .../StandardNiFiWebConfigurationContext.java    |  35 +--
 .../apache/nifi/web/StandardNiFiWebContext.java |   8 +-
 .../nifi/web/api/ApplicationResource.java       |  34 ++-
 .../nifi/web/api/BulletinBoardResource.java     |   7 +-
 .../apache/nifi/web/api/ClusterResource.java    |   6 +-
 .../apache/nifi/web/api/ConnectionResource.java |  80 ++----
 .../apache/nifi/web/api/ControllerResource.java | 113 +++-----
 .../nifi/web/api/ControllerServiceResource.java | 132 ++++------
 .../org/apache/nifi/web/api/FunnelResource.java |  43 ++-
 .../apache/nifi/web/api/HistoryResource.java    |  52 ++--
 .../apache/nifi/web/api/InputPortResource.java  |  48 ++--
 .../org/apache/nifi/web/api/LabelResource.java  |  46 ++--
 .../org/apache/nifi/web/api/NodeResource.java   |  20 +-
 .../apache/nifi/web/api/OutputPortResource.java |  48 ++--
 .../nifi/web/api/ProcessGroupResource.java      | 155 ++++-------
 .../apache/nifi/web/api/ProcessorResource.java  |  69 ++---
 .../apache/nifi/web/api/ProvenanceResource.java | 129 +++------
 .../web/api/RemoteProcessGroupResource.java     |  99 +++----
 .../nifi/web/api/ReportingTaskResource.java     |  92 ++-----
 .../apache/nifi/web/api/SnippetResource.java    |  79 +++---
 .../nifi/web/api/SystemDiagnosticsResource.java |   4 +-
 .../apache/nifi/web/api/TemplateResource.java   |  38 ++-
 .../apache/nifi/web/api/UserGroupResource.java  |  44 ++--
 .../org/apache/nifi/web/api/UserResource.java   |  33 +--
 .../config/AdministrationExceptionMapper.java   |   1 -
 ...ationCredentialsNotFoundExceptionMapper.java |   3 +-
 .../web/api/config/ClusterExceptionMapper.java  |   1 -
 .../NoResponseFromNodesExceptionMapper.java     |   3 +-
 .../nifi/web/api/config/ThrowableMapper.java    |   1 -
 .../org/apache/nifi/web/api/dto/DtoFactory.java | 172 ++++++------
 .../org/apache/nifi/web/api/package-info.java   |  41 +--
 .../request/BulletinBoardPatternParameter.java  |   3 +-
 .../nifi/web/api/request/ClientIdParameter.java |   3 +-
 .../nifi/web/api/request/DateTimeParameter.java |   3 +-
 .../nifi/web/api/request/DoubleParameter.java   |   3 +-
 .../nifi/web/api/request/IntegerParameter.java  |   3 +-
 .../nifi/web/api/request/LongParameter.java     |   3 +-
 .../ApplicationStartupContextListener.java      |   8 +-
 .../nifi/web/controller/ControllerFacade.java   | 162 ++++++------
 .../nifi/web/dao/ControllerServiceDAO.java      |   3 +-
 .../java/org/apache/nifi/web/dao/PortDAO.java   |   2 +-
 .../apache/nifi/web/dao/ProcessGroupDAO.java    |  11 +-
 .../org/apache/nifi/web/dao/ProcessorDAO.java   |   3 +-
 .../nifi/web/dao/RemoteProcessGroupDAO.java     |   4 +-
 .../apache/nifi/web/dao/ReportingTaskDAO.java   |  11 +-
 .../org/apache/nifi/web/dao/SnippetDAO.java     |   3 +-
 .../org/apache/nifi/web/dao/TemplateDAO.java    |  17 +-
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |  21 +-
 .../web/dao/impl/StandardConnectionDAO.java     |  48 ----
 .../dao/impl/StandardControllerServiceDAO.java  |  67 +----
 .../nifi/web/dao/impl/StandardFunnelDAO.java    |  48 ----
 .../nifi/web/dao/impl/StandardInputPortDAO.java |  41 ---
 .../nifi/web/dao/impl/StandardLabelDAO.java     |  48 ----
 .../web/dao/impl/StandardOutputPortDAO.java     |  41 ---
 .../web/dao/impl/StandardProcessGroupDAO.java   |  34 ---
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  49 ----
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  29 +--
 .../web/dao/impl/StandardReportingTaskDAO.java  |  63 +----
 .../nifi/web/dao/impl/StandardSnippetDAO.java   |  43 ---
 .../nifi/web/filter/NodeRequestFilter.java      |  21 +-
 .../org/apache/nifi/web/util/Availability.java  |   3 +-
 .../org/apache/nifi/web/util/SnippetUtils.java  |  12 +-
 .../apache/nifi/integration/NiFiWebApiTest.java |  27 --
 .../accesscontrol/AdminAccessControlTest.java   | 162 +++---------
 .../accesscontrol/DfmAccessControlTest.java     | 140 ++--------
 .../ReadOnlyAccessControlTest.java              | 162 +++---------
 .../util/NiFiTestAuthorizationProvider.java     |  36 +--
 .../nifi/integration/util/NiFiTestServer.java   |   7 +-
 .../nifi/integration/util/NiFiTestUser.java     |  88 ++++---
 .../nifi/web/docs/DocumentationController.java  |   2 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |  17 --
 .../org/apache/nifi/processors/GeoEnrichIP.java |  14 +-
 .../nifi/processors/maxmind/DatabaseReader.java |  40 +--
 329 files changed, 2277 insertions(+), 4857 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3e767fbd/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3e767fbd/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java
----------------------------------------------------------------------


[2/7] incubator-nifi git commit: NIFI-533: Fixed code to conform to checkstyle

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
index 1dd5b91,361f1ed..f7894d9
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java
@@@ -58,13 -58,18 +58,15 @@@ import org.apache.nifi.processor.except
  import org.apache.nifi.processor.util.StandardValidators;
  import org.apache.nifi.util.StopWatch;
  
 -/**
 - * This processor reads files from HDFS into NiFi FlowFiles.
 - */
  @TriggerWhenEmpty
  @Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"})
 -@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles")
 +@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles. This Processor will delete the file from HDFS after fetching it.")
  @WritesAttributes({
-         @WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."),
-         @WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".") })
+     @WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."),
 -    @WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if "
 -            + "the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse "
 -            + "Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".")})
 -@SeeAlso(PutHDFS.class)
++    @WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if the Directory property "
++            + "is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and "
++            + "a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".") })
 +@SeeAlso({PutHDFS.class, ListHDFS.class})
  public class GetHDFS extends AbstractHadoopProcessor {
  
      public static final String BUFFER_SIZE_KEY = "io.file.buffer.size";
@@@ -73,105 -78,101 +75,101 @@@
  
      // relationships
      public static final Relationship REL_SUCCESS = new Relationship.Builder()
--            .name("success")
--            .description("All files retrieved from HDFS are transferred to this relationship")
--            .build();
++    .name("success")
++    .description("All files retrieved from HDFS are transferred to this relationship")
++    .build();
  
      public static final Relationship REL_PASSTHROUGH = new Relationship.Builder()
--            .name("passthrough")
--            .description(
--                    "If this processor has an input queue for some reason, then FlowFiles arriving on that input are transferred to this relationship")
++    .name("passthrough")
++    .description(
++            "If this processor has an input queue for some reason, then FlowFiles arriving on that input are transferred to this relationship")
              .build();
  
      // properties
      public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
--            .name(DIRECTORY_PROP_NAME)
--            .description("The HDFS directory from which files should be read")
--            .required(true)
--            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
--            .build();
++    .name(DIRECTORY_PROP_NAME)
++    .description("The HDFS directory from which files should be read")
++    .required(true)
++    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
++    .build();
  
      public static final PropertyDescriptor RECURSE_SUBDIRS = new PropertyDescriptor.Builder()
--            .name("Recurse Subdirectories")
--            .description("Indicates whether to pull files from subdirectories of the HDFS directory")
--            .required(true)
--            .allowableValues("true", "false")
--            .defaultValue("true")
--            .build();
++    .name("Recurse Subdirectories")
++    .description("Indicates whether to pull files from subdirectories of the HDFS directory")
++    .required(true)
++    .allowableValues("true", "false")
++    .defaultValue("true")
++    .build();
  
      public static final PropertyDescriptor KEEP_SOURCE_FILE = new PropertyDescriptor.Builder()
--            .name("Keep Source File")
-             .description("Determines whether to delete the file from HDFS after it has been successfully transferred. If true, the file will be fetched repeatedly. This is intended for testing only.")
 -            .description("Determines whether to delete the file from HDFS after it has been successfully transferred")
--            .required(true)
--            .allowableValues("true", "false")
--            .defaultValue("false")
--            .build();
++    .name("Keep Source File")
++    .description("Determines whether to delete the file from HDFS after it has been successfully transferred. If true, the file will be fetched repeatedly. This is intended for testing only.")
++    .required(true)
++    .allowableValues("true", "false")
++    .defaultValue("false")
++    .build();
  
      public static final PropertyDescriptor FILE_FILTER_REGEX = new PropertyDescriptor.Builder()
--            .name("File Filter Regex")
-             .description(
-                     "A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular Expression will be fetched, otherwise all files will be fetched")
 -            .description("A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular "
 -                    + "Expression will be fetched, otherwise all files will be fetched")
++    .name("File Filter Regex")
++    .description("A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular "
++            + "Expression will be fetched, otherwise all files will be fetched")
              .required(false)
              .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
              .build();
  
      public static final PropertyDescriptor FILTER_MATCH_NAME_ONLY = new PropertyDescriptor.Builder()
--            .name("Filter Match Name Only")
-             .description(
-                     "If true then File Filter Regex will match on just the filename, otherwise subdirectory names will be included with filename in the regex comparison")
 -            .description("If true then File Filter Regex will match on just the filename, otherwise subdirectory names will be included with filename "
 -                    + "in the regex comparison")
++    .name("Filter Match Name Only")
++    .description("If true then File Filter Regex will match on just the filename, otherwise subdirectory names will be included with filename "
++            + "in the regex comparison")
              .required(true)
              .allowableValues("true", "false")
              .defaultValue("true")
              .build();
  
      public static final PropertyDescriptor IGNORE_DOTTED_FILES = new PropertyDescriptor.Builder()
--            .name("Ignore Dotted Files")
--            .description("If true, files whose names begin with a dot (\".\") will be ignored")
--            .required(true)
--            .allowableValues("true", "false")
--            .defaultValue("true")
--            .build();
++    .name("Ignore Dotted Files")
++    .description("If true, files whose names begin with a dot (\".\") will be ignored")
++    .required(true)
++    .allowableValues("true", "false")
++    .defaultValue("true")
++    .build();
  
      public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
--            .name("Minimum File Age")
-             .description(
-                     "The minimum age that a file must be in order to be pulled; any file younger than this amount of time (based on last modification date) will be ignored")
 -            .description("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (based on last modification date) will be ignored")
--            .required(true)
-             .addValidator(
-                     StandardValidators.createTimePeriodValidator(0, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
 -            .addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
--            .defaultValue("0 sec")
--            .build();
++    .name("Minimum File Age")
++    .description("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (based on last modification date) will be ignored")
++    .required(true)
++    .addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
++    .defaultValue("0 sec")
++    .build();
  
      public static final PropertyDescriptor MAX_AGE = new PropertyDescriptor.Builder()
--            .name("Maximum File Age")
-             .description(
-                     "The maximum age that a file must be in order to be pulled; any file older than this amount of time (based on last modification date) will be ignored")
 -            .description("The maximum age that a file must be in order to be pulled; any file older than this amount of time (based on last modification date) will be ignored")
--            .required(false)
-             .addValidator(
-                     StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
 -            .addValidator(StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
--            .build();
++    .name("Maximum File Age")
++    .description("The maximum age that a file must be in order to be pulled; any file older than this amount of time (based on last modification date) will be ignored")
++    .required(false)
++    .addValidator(StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
++    .build();
  
      public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
--            .name("Batch Size")
--            .description("The maximum number of files to pull in each iteration, based on run schedule.")
--            .required(true)
--            .defaultValue("100")
--            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
--            .build();
++    .name("Batch Size")
++    .description("The maximum number of files to pull in each iteration, based on run schedule.")
++    .required(true)
++    .defaultValue("100")
++    .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
++    .build();
  
      public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
--            .name("Polling Interval")
--            .description("Indicates how long to wait between performing directory listings")
--            .required(true)
--            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
--            .defaultValue("0 sec")
--            .build();
++    .name("Polling Interval")
++    .description("Indicates how long to wait between performing directory listings")
++    .required(true)
++    .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
++    .defaultValue("0 sec")
++    .build();
  
      public static final PropertyDescriptor BUFFER_SIZE = new PropertyDescriptor.Builder()
--            .name("IO Buffer Size")
--            .description("Amount of memory to use to buffer file contents during IO. This overrides the Hadoop Configuration")
--            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
--            .build();
++    .name("IO Buffer Size")
++    .description("Amount of memory to use to buffer file contents during IO. This overrides the Hadoop Configuration")
++    .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
++    .build();
  
      private static final Set<Relationship> relationships;
      protected static final List<PropertyDescriptor> localProperties;
@@@ -461,11 -463,35 +460,8 @@@
          return files;
      }
  
-     
- 
      /**
-      * Holder for a snapshot in time of some processor properties that are
-      * passed around.
 -     * Returns the relative path of the child that does not include the filename or the root path.
 -     *
 -     * @param root root
 -     * @param child child
 -     * @return the relative path of the child that does not include the filename or the root path
 -     */
 -    public static String getPathDifference(final Path root, final Path child) {
 -        final int depthDiff = child.depth() - root.depth();
 -        if (depthDiff <= 1) {
 -            return "".intern();
 -        }
 -        String lastRoot = root.getName();
 -        Path childsParent = child.getParent();
 -        final StringBuilder builder = new StringBuilder();
 -        builder.append(childsParent.getName());
 -        for (int i = (depthDiff - 3); i >= 0; i--) {
 -            childsParent = childsParent.getParent();
 -            String name = childsParent.getName();
 -            if (name.equals(lastRoot) && childsParent.toString().endsWith(root.toString())) {
 -                break;
 -            }
 -            builder.insert(0, Path.SEPARATOR).insert(0, name);
 -        }
 -        return builder.toString();
 -    }
 -
 -    /**
+      * Holder for a snapshot in time of some processor properties that are passed around.
       */
      protected static class ProcessorConfiguration {
  

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
index 707b50d,0000000..56a128a
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/ListHDFS.java
@@@ -1,466 -1,0 +1,469 @@@
 +/*
 + * 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.hadoop;
 +
 +import java.io.File;
 +import java.io.FileInputStream;
 +import java.io.FileOutputStream;
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.Date;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.permission.FsAction;
 +import org.apache.hadoop.fs.permission.FsPermission;
 +import org.apache.nifi.annotation.behavior.TriggerSerially;
 +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
 +import org.apache.nifi.annotation.behavior.WritesAttribute;
 +import org.apache.nifi.annotation.behavior.WritesAttributes;
 +import org.apache.nifi.annotation.documentation.CapabilityDescription;
 +import org.apache.nifi.annotation.documentation.SeeAlso;
 +import org.apache.nifi.annotation.documentation.Tags;
 +import org.apache.nifi.annotation.notification.OnPrimaryNodeStateChange;
 +import org.apache.nifi.annotation.notification.PrimaryNodeState;
 +import org.apache.nifi.components.PropertyDescriptor;
 +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
 +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.ProcessorInitializationContext;
 +import org.apache.nifi.processor.Relationship;
 +import org.apache.nifi.processor.exception.ProcessException;
 +import org.apache.nifi.processor.util.StandardValidators;
 +import org.apache.nifi.processors.hadoop.util.HDFSListing;
 +import org.apache.nifi.processors.hadoop.util.StringSerDe;
 +import org.codehaus.jackson.JsonGenerationException;
 +import org.codehaus.jackson.JsonNode;
 +import org.codehaus.jackson.JsonParseException;
 +import org.codehaus.jackson.map.JsonMappingException;
 +import org.codehaus.jackson.map.ObjectMapper;
 +
 +
 +@TriggerSerially
 +@TriggerWhenEmpty
 +@Tags({"hadoop", "HDFS", "get", "list", "ingest", "source", "filesystem"})
 +@CapabilityDescription("Retrieves a listing of files from HDFS. For each file that is listed in HDFS, creates a FlowFile that represents "
- 		+ "the HDFS file so that it can be fetched in conjunction with ListHDFS. This Processor is designed to run on Primary Node only "
- 		+ "in a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left off without duplicating "
- 		+ "all of the data. Unlike GetHDFS, this Processor does not delete any data from HDFS.")
++        + "the HDFS file so that it can be fetched in conjunction with ListHDFS. This Processor is designed to run on Primary Node only "
++        + "in a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left off without duplicating "
++        + "all of the data. Unlike GetHDFS, this Processor does not delete any data from HDFS.")
 +@WritesAttributes({
-         @WritesAttribute(attribute="filename", description="The name of the file that was read from HDFS."),
-         @WritesAttribute(attribute="path", description="The path is set to the absolute path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"/tmp/abc/1/2/3\"."),
- 		@WritesAttribute(attribute="hdfs.owner", description="The user that owns the file in HDFS"),
- 		@WritesAttribute(attribute="hdfs.group", description="The group that owns the file in HDFS"),
- 		@WritesAttribute(attribute="hdfs.lastModified", description="The timestamp of when the file in HDFS was last modified, as milliseconds since midnight Jan 1, 1970 UTC"),
- 		@WritesAttribute(attribute="hdfs.length", description="The number of bytes in the file in HDFS"),
- 		@WritesAttribute(attribute="hdfs.replication", description="The number of HDFS replicas for hte file"),
- 		@WritesAttribute(attribute="hdfs.permissions", description="The permissions for the file in HDFS. This is formatted as 3 characters for the owner, 3 for the group, and 3 for other users. For example rw-rw-r--")
++    @WritesAttribute(attribute="filename", description="The name of the file that was read from HDFS."),
++    @WritesAttribute(attribute="path", description="The path is set to the absolute path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, "
++            + "then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up "
++            + "from /tmp/abc/1/2/3, then the path attribute will be set to \"/tmp/abc/1/2/3\"."),
++    @WritesAttribute(attribute="hdfs.owner", description="The user that owns the file in HDFS"),
++    @WritesAttribute(attribute="hdfs.group", description="The group that owns the file in HDFS"),
++    @WritesAttribute(attribute="hdfs.lastModified", description="The timestamp of when the file in HDFS was last modified, as milliseconds since midnight Jan 1, 1970 UTC"),
++    @WritesAttribute(attribute="hdfs.length", description="The number of bytes in the file in HDFS"),
++    @WritesAttribute(attribute="hdfs.replication", description="The number of HDFS replicas for hte file"),
++    @WritesAttribute(attribute="hdfs.permissions", description="The permissions for the file in HDFS. This is formatted as 3 characters for the owner, "
++            + "3 for the group, and 3 for other users. For example rw-rw-r--")
 +})
 +@SeeAlso({GetHDFS.class, FetchHDFS.class, PutHDFS.class})
 +public class ListHDFS extends AbstractHadoopProcessor {
- 
- 	public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
- 		.name("Distributed Cache Service")
- 		.description("Specifies the Controller Service that should be used to maintain state about what has been pulled from HDFS so that if a new node begins pulling data, it won't duplicate all of the work that has been done.")
- 		.required(true)
- 		.identifiesControllerService(DistributedMapCacheClient.class)
- 		.build();
++    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
++        .name("Distributed Cache Service")
++        .description("Specifies the Controller Service that should be used to maintain state about what has been pulled from HDFS so that if a new node "
++                + "begins pulling data, it won't duplicate all of the work that has been done.")
++        .required(true)
++        .identifiesControllerService(DistributedMapCacheClient.class)
++        .build();
 +
 +    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
- 	    .name(DIRECTORY_PROP_NAME)
- 	    .description("The HDFS directory from which files should be read")
- 	    .required(true)
- 	    .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- 	    .build();
- 	
- 	public static final PropertyDescriptor RECURSE_SUBDIRS = new PropertyDescriptor.Builder()
- 	    .name("Recurse Subdirectories")
- 	    .description("Indicates whether to list files from subdirectories of the HDFS directory")
- 	    .required(true)
- 	    .allowableValues("true", "false")
- 	    .defaultValue("true")
- 	    .build();
- 	
- 	
++        .name(DIRECTORY_PROP_NAME)
++        .description("The HDFS directory from which files should be read")
++        .required(true)
++        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
++        .build();
++
++    public static final PropertyDescriptor RECURSE_SUBDIRS = new PropertyDescriptor.Builder()
++        .name("Recurse Subdirectories")
++        .description("Indicates whether to list files from subdirectories of the HDFS directory")
++        .required(true)
++        .allowableValues("true", "false")
++        .defaultValue("true")
++        .build();
++
++
 +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
- 	    .name("success")
- 	    .description("All FlowFiles are transferred to this relationship")
- 	    .build();
++        .name("success")
++        .description("All FlowFiles are transferred to this relationship")
++        .build();
 +
 +    private volatile Long lastListingTime = null;
 +    private volatile Set<Path> latestPathsListed = new HashSet<>();
 +    private volatile boolean electedPrimaryNode = false;
 +    private File persistenceFile = null;
-     
++
 +    @Override
 +    protected void init(final ProcessorInitializationContext context) {
-     	super.init(context);
-     	persistenceFile = new File("conf/state/" + getIdentifier());
++        super.init(context);
++        persistenceFile = new File("conf/state/" + getIdentifier());
++    }
++
++    @Override
++    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
++        final List<PropertyDescriptor> properties = new ArrayList<>();
++        properties.add(HADOOP_CONFIGURATION_RESOURCES);
++        properties.add(DISTRIBUTED_CACHE_SERVICE);
++        properties.add(DIRECTORY);
++        properties.add(RECURSE_SUBDIRS);
++        return properties;
++    }
++
++    @Override
++    public Set<Relationship> getRelationships() {
++        final Set<Relationship> relationships = new HashSet<>();
++        relationships.add(REL_SUCCESS);
++        return relationships;
++    }
++
++    private String getKey(final String directory) {
++        return getIdentifier() + ".lastListingTime." + directory;
++    }
++
++    @OnPrimaryNodeStateChange
++    public void onPrimaryNodeChange(final PrimaryNodeState newState) {
++        if ( newState == PrimaryNodeState.ELECTED_PRIMARY_NODE ) {
++            electedPrimaryNode = true;
++        }
++    }
++
++    @Override
++    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
++        if ( descriptor.equals(DIRECTORY) ) {
++            lastListingTime = null; // clear lastListingTime so that we have to fetch new time
++            latestPathsListed = new HashSet<>();
++        }
 +    }
- 	
- 	@Override
- 	protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
- 		final List<PropertyDescriptor> properties = new ArrayList<>();
- 		properties.add(HADOOP_CONFIGURATION_RESOURCES);
- 		properties.add(DISTRIBUTED_CACHE_SERVICE);
- 		properties.add(DIRECTORY);
- 		properties.add(RECURSE_SUBDIRS);
- 		return properties;
- 	}
- 	
- 	@Override
- 	public Set<Relationship> getRelationships() {
- 		final Set<Relationship> relationships = new HashSet<>();
- 		relationships.add(REL_SUCCESS);
- 		return relationships;
- 	}
- 
- 	private String getKey(final String directory) {
- 		return getIdentifier() + ".lastListingTime." + directory;
- 	}
- 	
- 	@OnPrimaryNodeStateChange
- 	public void onPrimaryNodeChange(final PrimaryNodeState newState) {
- 		if ( newState == PrimaryNodeState.ELECTED_PRIMARY_NODE ) {
- 			electedPrimaryNode = true;
- 		}
- 	}
- 	
- 	@Override
- 	public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
- 		if ( descriptor.equals(DIRECTORY) ) {
- 			lastListingTime = null;	// clear lastListingTime so that we have to fetch new time
- 			latestPathsListed = new HashSet<>();
- 		}
- 	}
- 	
- 	private HDFSListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException {
- 		final ObjectMapper mapper = new ObjectMapper();
++
++    private HDFSListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException {
++        final ObjectMapper mapper = new ObjectMapper();
 +        final JsonNode jsonNode = mapper.readTree(serializedState);
 +        return mapper.readValue(jsonNode, HDFSListing.class);
- 	}
- 	
- 	
- 	@Override
- 	public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
- 		final String directory = context.getProperty(DIRECTORY).getValue();
- 
- 		// Determine the timestamp for the last file that we've listed.
- 		Long minTimestamp = lastListingTime;
- 		if ( minTimestamp == null || electedPrimaryNode ) {
- 			// We haven't yet restored any state from local or distributed state - or it's been at least a minute since
- 			// we have performed a listing. In this case, 
- 			// First, attempt to get timestamp from distributed cache service.
- 			final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
- 			
- 			try {
- 				final StringSerDe serde = new StringSerDe();
- 				final String serializedState = client.get(getKey(directory), serde, serde);
- 				if ( serializedState == null || serializedState.isEmpty() ) {
- 					minTimestamp = null;
- 					this.latestPathsListed = Collections.emptySet();
- 				} else {
- 					final HDFSListing listing = deserialize(serializedState);
- 					this.lastListingTime = listing.getLatestTimestamp().getTime();
- 					minTimestamp = listing.getLatestTimestamp().getTime();
- 					this.latestPathsListed = listing.toPaths();
- 				}
- 				
- 				this.lastListingTime = minTimestamp;
- 				electedPrimaryNode = false;	// no requirement to pull an update from the distributed cache anymore.
- 			} catch (final IOException ioe) {
- 				getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished.");
- 				context.yield();
- 				return;
- 			}
- 			
- 			// Check the persistence file. We want to use the latest timestamp that we have so that
- 			// we don't duplicate data.
- 			try {
- 				if ( persistenceFile.exists() ) {
- 					try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
- 						final Properties props = new Properties();
- 						props.load(fis);
- 
- 						// get the local timestamp for this directory, if it exists.
- 						final String locallyPersistedValue = props.getProperty(directory);
- 						if ( locallyPersistedValue != null ) {
- 							final HDFSListing listing = deserialize(locallyPersistedValue);
- 							final long localTimestamp = listing.getLatestTimestamp().getTime();
- 							
- 							// If distributed state doesn't have an entry or the local entry is later than the distributed state,
- 							// update the distributed state so that we are in sync.
- 							if (minTimestamp == null || localTimestamp > minTimestamp) {
- 								minTimestamp = localTimestamp;
- 								
- 								// Our local persistence file shows a later time than the Distributed service.
- 								// Update the distributed service to match our local state.
- 								try {
- 									final StringSerDe serde = new StringSerDe();
- 									client.put(getKey(directory), locallyPersistedValue, serde, serde);
- 								} catch (final IOException ioe) {
- 									getLogger().warn("Local timestamp for {} is {}, which is later than Distributed state but failed to update Distributed "
- 											+ "state due to {}. If a new node performs HDFS Listing, data duplication may occur", 
- 											new Object[] {directory, locallyPersistedValue, ioe});
- 								}
- 							}
- 						}
- 					}
- 				}
- 			} catch (final IOException ioe) {
- 				getLogger().warn("Failed to recover local state due to {}. Assuming that the state from the distributed cache is correct.", ioe);
- 			}
- 		}
- 		
- 		
- 		// Pull in any file that is newer than the timestamp that we have.
- 		final FileSystem hdfs = hdfsResources.get().getValue();
- 		final boolean recursive = context.getProperty(RECURSE_SUBDIRS).asBoolean();
- 		final Path rootPath = new Path(directory);
- 		
- 		int listCount = 0;
- 		Long latestListingModTime = null;
- 		final Set<FileStatus> statuses;
- 		try {
- 			statuses = getStatuses(rootPath, recursive, hdfs);
- 			for ( final FileStatus status : statuses ) {
- 				// don't get anything where the last modified timestamp is equal to our current timestamp.
- 				// if we do, then we run the risk of multiple files having the same last mod date but us only
- 				// seeing a portion of them.
- 				// I.e., there could be 5 files with last mod date = (now). But if we do the listing now, maybe
- 				// only 2 exist and 3 more will exist later in this millisecond. So we ignore anything with a
- 				// modified date not before the current time.
- 				final long fileModTime = status.getModificationTime();
- 				
- 				// we only want the file if its timestamp is later than the minTimestamp or equal to and we didn't pull it last time.
- 				// Also, HDFS creates files with the suffix _COPYING_ when they are being written - we want to ignore those.
- 				boolean fetch = !status.getPath().getName().endsWith("_COPYING_") &&
- 						(minTimestamp == null || fileModTime > minTimestamp || (fileModTime == minTimestamp && !latestPathsListed.contains(status.getPath())));
- 				
- 				// Create the FlowFile for this path.
- 				if ( fetch ) {
- 					final Map<String, String> attributes = createAttributes(status);
- 					FlowFile flowFile = session.create();
- 					flowFile = session.putAllAttributes(flowFile, attributes);
- 					session.transfer(flowFile, REL_SUCCESS);
- 					listCount++;
- 					
- 					if ( latestListingModTime == null || fileModTime > latestListingModTime ) {
- 						latestListingModTime = fileModTime;
- 					}
- 				}
- 			}
- 		} catch (final IOException ioe) {
- 			getLogger().error("Failed to perform listing of HDFS due to {}", new Object[] {ioe});
- 			return;
- 		}
- 		
- 		if ( listCount > 0 ) {
- 			getLogger().info("Successfully created listing with {} new files from HDFS", new Object[] {listCount});
- 			session.commit();
- 
- 			// We have performed a listing and pushed the FlowFiles out.
- 			// Now, we need to persist state about the Last Modified timestamp of the newest file
- 			// that we pulled in. We do this in order to avoid pulling in the same file twice.
- 			// However, we want to save the state both locally and remotely.
- 			// We store the state remotely so that if a new Primary Node is chosen, it can pick up where the
- 			// previously Primary Node left off.
- 			// We also store the state locally so that if the node is restarted, and the node cannot contact
- 			// the distributed state cache, the node can continue to run (if it is primary node).
- 			String serializedState = null;
- 			try {
- 				serializedState = serializeState(latestListingModTime, statuses);
- 			} catch (final Exception e) {
- 				getLogger().error("Failed to serialize state due to {}", new Object[] {e});
- 			}
- 			
- 			if ( serializedState != null ) {
- 				// Save our state locally.
- 				try {
- 					persistLocalState(directory, serializedState);
- 				} catch (final IOException ioe) {
- 					getLogger().warn("Unable to save state locally. If the node is restarted now, data may be duplicated. Failure is due to {}", ioe);
- 				}
- 	
- 				// Attempt to save state to remote server.
- 				final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
- 				try {
- 					client.put(getKey(directory), serializedState, new StringSerDe(), new StringSerDe());
- 				} catch (final IOException ioe) {
- 					getLogger().warn("Unable to communicate with distributed cache server due to {}. Persisting state locally instead.", ioe);
- 				}
- 			}
- 			
- 			lastListingTime = latestListingModTime;
- 		} else {
- 			getLogger().debug("There is no data to list. Yielding.");
- 			context.yield();
- 			
- 			// lastListingTime = 0 so that we don't continually poll the distributed cache / local file system
- 			if ( lastListingTime == null ) {
- 				lastListingTime = 0L;
- 			}
- 			
- 			return;
- 		}
- 	}
- 	
- 	private Set<FileStatus> getStatuses(final Path path, final boolean recursive, final FileSystem hdfs) throws IOException {
- 		final Set<FileStatus> statusSet = new HashSet<>();
- 		
- 		final FileStatus[] statuses = hdfs.listStatus(path);
- 		
- 		for ( final FileStatus status : statuses ) {
- 			if ( status.isDirectory() ) {
- 				if ( recursive ) {
- 					try {
- 						statusSet.addAll(getStatuses(status.getPath(), recursive, hdfs));
- 					} catch (final IOException ioe) {
- 						getLogger().error("Failed to retrieve HDFS listing for subdirectory {} due to {}; will continue listing others", new Object[] {status.getPath(), ioe});
- 					}
- 				}
- 			} else {
- 				statusSet.add(status);
- 			}
- 		}
- 		
- 		return statusSet;
- 	}
- 	
- 	
- 	private String serializeState(final long latestListingTime, final Set<FileStatus> statuses) throws JsonGenerationException, JsonMappingException, IOException {
- 		// we need to keep track of all files that we pulled in that had a modification time equal to
- 		// lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
- 		// that have a mod time equal to that timestamp because more files may come in with the same timestamp
- 		// later in the same millisecond.
- 		if ( statuses.isEmpty() ) {
- 			return null;
- 		} else {
- 			final List<FileStatus> sortedStatuses = new ArrayList<>(statuses);
- 			Collections.sort(sortedStatuses, new Comparator<FileStatus>() {
- 				@Override
- 				public int compare(final FileStatus o1, final FileStatus o2) {
- 					return Long.compare(o1.getModificationTime(), o2.getModificationTime());
- 				}
- 			});
- 			
- 			final long latestListingModTime = sortedStatuses.get(sortedStatuses.size() - 1).getModificationTime();
- 			final Set<Path> pathsWithModTimeEqualToListingModTime = new HashSet<>();
- 			for (int i=sortedStatuses.size() - 1; i >= 0; i--) {
- 				final FileStatus status = sortedStatuses.get(i);
- 				if (status.getModificationTime() == latestListingModTime) {
- 					pathsWithModTimeEqualToListingModTime.add(status.getPath());
- 				}
- 			}
- 			
- 			this.latestPathsListed = pathsWithModTimeEqualToListingModTime;
- 			
- 			final HDFSListing listing = new HDFSListing();
- 			listing.setLatestTimestamp(new Date(latestListingModTime));
- 			final Set<String> paths = new HashSet<>();
- 			for ( final Path path : pathsWithModTimeEqualToListingModTime ) {
- 				paths.add(path.toUri().toString());
- 			}
- 			listing.setMatchingPaths(paths);
- 
- 			final ObjectMapper mapper = new ObjectMapper();
- 			final String serializedState = mapper.writerWithType(HDFSListing.class).writeValueAsString(listing);
- 			return serializedState;
- 		}
- 	}
- 	
- 	private void persistLocalState(final String directory, final String serializedState) throws IOException {
- 		// we need to keep track of all files that we pulled in that had a modification time equal to
- 		// lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
- 		// that have a mod time equal to that timestamp because more files may come in with the same timestamp
- 		// later in the same millisecond.
- 		final File dir = persistenceFile.getParentFile();
- 		if ( !dir.exists() && !dir.mkdirs() ) {
- 			throw new IOException("Could not create directory " + dir.getAbsolutePath() + " in order to save local state");
- 		}
- 		
- 		final Properties props = new Properties();
- 		if ( persistenceFile.exists() ) {
- 			try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
- 				props.load(fis);
- 			}
- 		}
- 
- 		props.setProperty(directory, serializedState);
- 		
- 		try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) {
- 			props.store(fos, null);
- 		}
- 	}
- 
- 	private String getAbsolutePath(final Path path) {
- 		final Path parent = path.getParent();
- 		final String prefix = (parent == null || parent.getName().equals("")) ? "" : getAbsolutePath(parent);
- 		return prefix + "/" + path.getName();
- 	}
- 	
- 	private Map<String, String> createAttributes(final FileStatus status) {
- 		final Map<String, String> attributes = new HashMap<>();
- 		attributes.put(CoreAttributes.FILENAME.key(), status.getPath().getName());
- 		attributes.put(CoreAttributes.PATH.key(), getAbsolutePath(status.getPath().getParent()));
- 		
- 		attributes.put("hdfs.owner", status.getOwner());
- 		attributes.put("hdfs.group", status.getGroup());
- 		attributes.put("hdfs.lastModified", String.valueOf(status.getModificationTime()));
- 		attributes.put("hdfs.length", String.valueOf(status.getLen()));
- 		attributes.put("hdfs.replication", String.valueOf(status.getReplication()));
- 		
- 		final FsPermission permission = status.getPermission();
- 		final String perms = getPerms(permission.getUserAction()) + getPerms(permission.getGroupAction()) + getPerms(permission.getOtherAction());
- 		attributes.put("hdfs.permissions", perms);
- 		return attributes;
- 	}
- 	
- 	private String getPerms(final FsAction action) {
- 		final StringBuilder sb = new StringBuilder();
- 		if ( action.implies(FsAction.READ) ) {
- 			sb.append("r");
- 		} else {
- 			sb.append("-");
- 		}
- 		
- 		if ( action.implies(FsAction.WRITE) ) {
- 			sb.append("w");
- 		} else {
- 			sb.append("-");
- 		}
- 		
- 		if ( action.implies(FsAction.EXECUTE) ) {
- 			sb.append("x");
- 		} else {
- 			sb.append("-");
- 		}
- 		
- 		return sb.toString();
- 	}
++    }
++
++
++    @Override
++    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
++        final String directory = context.getProperty(DIRECTORY).getValue();
++
++        // Determine the timestamp for the last file that we've listed.
++        Long minTimestamp = lastListingTime;
++        if ( minTimestamp == null || electedPrimaryNode ) {
++            // We haven't yet restored any state from local or distributed state - or it's been at least a minute since
++            // we have performed a listing. In this case,
++            // First, attempt to get timestamp from distributed cache service.
++            final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
++
++            try {
++                final StringSerDe serde = new StringSerDe();
++                final String serializedState = client.get(getKey(directory), serde, serde);
++                if ( serializedState == null || serializedState.isEmpty() ) {
++                    minTimestamp = null;
++                    this.latestPathsListed = Collections.emptySet();
++                } else {
++                    final HDFSListing listing = deserialize(serializedState);
++                    this.lastListingTime = listing.getLatestTimestamp().getTime();
++                    minTimestamp = listing.getLatestTimestamp().getTime();
++                    this.latestPathsListed = listing.toPaths();
++                }
++
++                this.lastListingTime = minTimestamp;
++                electedPrimaryNode = false; // no requirement to pull an update from the distributed cache anymore.
++            } catch (final IOException ioe) {
++                getLogger().error("Failed to retrieve timestamp of last listing from Distributed Cache Service. Will not perform listing until this is accomplished.");
++                context.yield();
++                return;
++            }
++
++            // Check the persistence file. We want to use the latest timestamp that we have so that
++            // we don't duplicate data.
++            try {
++                if ( persistenceFile.exists() ) {
++                    try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
++                        final Properties props = new Properties();
++                        props.load(fis);
++
++                        // get the local timestamp for this directory, if it exists.
++                        final String locallyPersistedValue = props.getProperty(directory);
++                        if ( locallyPersistedValue != null ) {
++                            final HDFSListing listing = deserialize(locallyPersistedValue);
++                            final long localTimestamp = listing.getLatestTimestamp().getTime();
++
++                            // If distributed state doesn't have an entry or the local entry is later than the distributed state,
++                            // update the distributed state so that we are in sync.
++                            if (minTimestamp == null || localTimestamp > minTimestamp) {
++                                minTimestamp = localTimestamp;
++
++                                // Our local persistence file shows a later time than the Distributed service.
++                                // Update the distributed service to match our local state.
++                                try {
++                                    final StringSerDe serde = new StringSerDe();
++                                    client.put(getKey(directory), locallyPersistedValue, serde, serde);
++                                } catch (final IOException ioe) {
++                                    getLogger().warn("Local timestamp for {} is {}, which is later than Distributed state but failed to update Distributed "
++                                            + "state due to {}. If a new node performs HDFS Listing, data duplication may occur",
++                                            new Object[] {directory, locallyPersistedValue, ioe});
++                                }
++                            }
++                        }
++                    }
++                }
++            } catch (final IOException ioe) {
++                getLogger().warn("Failed to recover local state due to {}. Assuming that the state from the distributed cache is correct.", ioe);
++            }
++        }
++
++
++        // Pull in any file that is newer than the timestamp that we have.
++        final FileSystem hdfs = hdfsResources.get().getValue();
++        final boolean recursive = context.getProperty(RECURSE_SUBDIRS).asBoolean();
++        final Path rootPath = new Path(directory);
++
++        int listCount = 0;
++        Long latestListingModTime = null;
++        final Set<FileStatus> statuses;
++        try {
++            statuses = getStatuses(rootPath, recursive, hdfs);
++            for ( final FileStatus status : statuses ) {
++                // don't get anything where the last modified timestamp is equal to our current timestamp.
++                // if we do, then we run the risk of multiple files having the same last mod date but us only
++                // seeing a portion of them.
++                // I.e., there could be 5 files with last mod date = (now). But if we do the listing now, maybe
++                // only 2 exist and 3 more will exist later in this millisecond. So we ignore anything with a
++                // modified date not before the current time.
++                final long fileModTime = status.getModificationTime();
++
++                // we only want the file if its timestamp is later than the minTimestamp or equal to and we didn't pull it last time.
++                // Also, HDFS creates files with the suffix _COPYING_ when they are being written - we want to ignore those.
++                boolean fetch = !status.getPath().getName().endsWith("_COPYING_")
++                        && (minTimestamp == null || fileModTime > minTimestamp || (fileModTime == minTimestamp && !latestPathsListed.contains(status.getPath())));
++
++                // Create the FlowFile for this path.
++                if ( fetch ) {
++                    final Map<String, String> attributes = createAttributes(status);
++                    FlowFile flowFile = session.create();
++                    flowFile = session.putAllAttributes(flowFile, attributes);
++                    session.transfer(flowFile, REL_SUCCESS);
++                    listCount++;
++
++                    if ( latestListingModTime == null || fileModTime > latestListingModTime ) {
++                        latestListingModTime = fileModTime;
++                    }
++                }
++            }
++        } catch (final IOException ioe) {
++            getLogger().error("Failed to perform listing of HDFS due to {}", new Object[] {ioe});
++            return;
++        }
++
++        if ( listCount > 0 ) {
++            getLogger().info("Successfully created listing with {} new files from HDFS", new Object[] {listCount});
++            session.commit();
++
++            // We have performed a listing and pushed the FlowFiles out.
++            // Now, we need to persist state about the Last Modified timestamp of the newest file
++            // that we pulled in. We do this in order to avoid pulling in the same file twice.
++            // However, we want to save the state both locally and remotely.
++            // We store the state remotely so that if a new Primary Node is chosen, it can pick up where the
++            // previously Primary Node left off.
++            // We also store the state locally so that if the node is restarted, and the node cannot contact
++            // the distributed state cache, the node can continue to run (if it is primary node).
++            String serializedState = null;
++            try {
++                serializedState = serializeState(latestListingModTime, statuses);
++            } catch (final Exception e) {
++                getLogger().error("Failed to serialize state due to {}", new Object[] {e});
++            }
++
++            if ( serializedState != null ) {
++                // Save our state locally.
++                try {
++                    persistLocalState(directory, serializedState);
++                } catch (final IOException ioe) {
++                    getLogger().warn("Unable to save state locally. If the node is restarted now, data may be duplicated. Failure is due to {}", ioe);
++                }
++
++                // Attempt to save state to remote server.
++                final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
++                try {
++                    client.put(getKey(directory), serializedState, new StringSerDe(), new StringSerDe());
++                } catch (final IOException ioe) {
++                    getLogger().warn("Unable to communicate with distributed cache server due to {}. Persisting state locally instead.", ioe);
++                }
++            }
++
++            lastListingTime = latestListingModTime;
++        } else {
++            getLogger().debug("There is no data to list. Yielding.");
++            context.yield();
++
++            // lastListingTime = 0 so that we don't continually poll the distributed cache / local file system
++            if ( lastListingTime == null ) {
++                lastListingTime = 0L;
++            }
++
++            return;
++        }
++    }
++
++    private Set<FileStatus> getStatuses(final Path path, final boolean recursive, final FileSystem hdfs) throws IOException {
++        final Set<FileStatus> statusSet = new HashSet<>();
++
++        final FileStatus[] statuses = hdfs.listStatus(path);
++
++        for ( final FileStatus status : statuses ) {
++            if ( status.isDirectory() ) {
++                if ( recursive ) {
++                    try {
++                        statusSet.addAll(getStatuses(status.getPath(), recursive, hdfs));
++                    } catch (final IOException ioe) {
++                        getLogger().error("Failed to retrieve HDFS listing for subdirectory {} due to {}; will continue listing others", new Object[] {status.getPath(), ioe});
++                    }
++                }
++            } else {
++                statusSet.add(status);
++            }
++        }
++
++        return statusSet;
++    }
++
++
++    private String serializeState(final long latestListingTime, final Set<FileStatus> statuses) throws JsonGenerationException, JsonMappingException, IOException {
++        // we need to keep track of all files that we pulled in that had a modification time equal to
++        // lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
++        // that have a mod time equal to that timestamp because more files may come in with the same timestamp
++        // later in the same millisecond.
++        if ( statuses.isEmpty() ) {
++            return null;
++        } else {
++            final List<FileStatus> sortedStatuses = new ArrayList<>(statuses);
++            Collections.sort(sortedStatuses, new Comparator<FileStatus>() {
++                @Override
++                public int compare(final FileStatus o1, final FileStatus o2) {
++                    return Long.compare(o1.getModificationTime(), o2.getModificationTime());
++                }
++            });
++
++            final long latestListingModTime = sortedStatuses.get(sortedStatuses.size() - 1).getModificationTime();
++            final Set<Path> pathsWithModTimeEqualToListingModTime = new HashSet<>();
++            for (int i=sortedStatuses.size() - 1; i >= 0; i--) {
++                final FileStatus status = sortedStatuses.get(i);
++                if (status.getModificationTime() == latestListingModTime) {
++                    pathsWithModTimeEqualToListingModTime.add(status.getPath());
++                }
++            }
++
++            this.latestPathsListed = pathsWithModTimeEqualToListingModTime;
++
++            final HDFSListing listing = new HDFSListing();
++            listing.setLatestTimestamp(new Date(latestListingModTime));
++            final Set<String> paths = new HashSet<>();
++            for ( final Path path : pathsWithModTimeEqualToListingModTime ) {
++                paths.add(path.toUri().toString());
++            }
++            listing.setMatchingPaths(paths);
++
++            final ObjectMapper mapper = new ObjectMapper();
++            final String serializedState = mapper.writerWithType(HDFSListing.class).writeValueAsString(listing);
++            return serializedState;
++        }
++    }
++
++    private void persistLocalState(final String directory, final String serializedState) throws IOException {
++        // we need to keep track of all files that we pulled in that had a modification time equal to
++        // lastListingTime so that we can avoid pulling those files in again. We can't just ignore any files
++        // that have a mod time equal to that timestamp because more files may come in with the same timestamp
++        // later in the same millisecond.
++        final File dir = persistenceFile.getParentFile();
++        if ( !dir.exists() && !dir.mkdirs() ) {
++            throw new IOException("Could not create directory " + dir.getAbsolutePath() + " in order to save local state");
++        }
++
++        final Properties props = new Properties();
++        if ( persistenceFile.exists() ) {
++            try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
++                props.load(fis);
++            }
++        }
++
++        props.setProperty(directory, serializedState);
++
++        try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) {
++            props.store(fos, null);
++        }
++    }
++
++    private String getAbsolutePath(final Path path) {
++        final Path parent = path.getParent();
++        final String prefix = (parent == null || parent.getName().equals("")) ? "" : getAbsolutePath(parent);
++        return prefix + "/" + path.getName();
++    }
++
++    private Map<String, String> createAttributes(final FileStatus status) {
++        final Map<String, String> attributes = new HashMap<>();
++        attributes.put(CoreAttributes.FILENAME.key(), status.getPath().getName());
++        attributes.put(CoreAttributes.PATH.key(), getAbsolutePath(status.getPath().getParent()));
++
++        attributes.put("hdfs.owner", status.getOwner());
++        attributes.put("hdfs.group", status.getGroup());
++        attributes.put("hdfs.lastModified", String.valueOf(status.getModificationTime()));
++        attributes.put("hdfs.length", String.valueOf(status.getLen()));
++        attributes.put("hdfs.replication", String.valueOf(status.getReplication()));
++
++        final FsPermission permission = status.getPermission();
++        final String perms = getPerms(permission.getUserAction()) + getPerms(permission.getGroupAction()) + getPerms(permission.getOtherAction());
++        attributes.put("hdfs.permissions", perms);
++        return attributes;
++    }
++
++    private String getPerms(final FsAction action) {
++        final StringBuilder sb = new StringBuilder();
++        if ( action.implies(FsAction.READ) ) {
++            sb.append("r");
++        } else {
++            sb.append("-");
++        }
++
++        if ( action.implies(FsAction.WRITE) ) {
++            sb.append("w");
++        } else {
++            sb.append("-");
++        }
++
++        if ( action.implies(FsAction.EXECUTE) ) {
++            sb.append("x");
++        } else {
++            sb.append("-");
++        }
++
++        return sb.toString();
++    }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
index 9f4d68b,0000000..49957f5
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/HDFSListing.java
@@@ -1,83 -1,0 +1,83 @@@
 +/*
 + * 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.hadoop.util;
 +
 +import java.util.Collection;
 +import java.util.Date;
 +import java.util.HashSet;
 +import java.util.Set;
 +
 +import javax.xml.bind.annotation.XmlTransient;
 +import javax.xml.bind.annotation.XmlType;
 +
 +import org.apache.hadoop.fs.Path;
 +
 +/**
 + * A simple POJO for maintaining state about the last HDFS Listing that was performed so that
-  * we can avoid pulling the same file multiple times 
++ * we can avoid pulling the same file multiple times
 + */
 +@XmlType(name = "listing")
 +public class HDFSListing {
- 	private Date latestTimestamp;
- 	private Collection<String> matchingPaths;
- 	
- 	/**
- 	 * @return the modification date of the newest file that was contained in the HDFS Listing
- 	 */
- 	public Date getLatestTimestamp() {
- 		return latestTimestamp;
- 	}
- 	
- 	/**
- 	 * Sets the timestamp of the modification date of the newest file that was contained in the HDFS Listing
- 	 * 
- 	 * @param latestTimestamp the timestamp of the modification date of the newest file that was contained in the HDFS Listing
- 	 */
- 	public void setLatestTimestamp(Date latestTimestamp) {
- 		this.latestTimestamp = latestTimestamp;
- 	}
- 	
- 	/**
- 	 * @return a Collection containing the paths of all files in the HDFS Listing whose Modification date
- 	 * was equal to {@link #getLatestTimestamp()}
- 	 */
- 	@XmlTransient
- 	public Collection<String> getMatchingPaths() {
- 		return matchingPaths;
- 	}
- 	
- 	/**
- 	 * @return a Collection of {@link Path} objects equivalent to those returned by {@link #getMatchingPaths()}
- 	 */
- 	public Set<Path> toPaths() {
- 		final Set<Path> paths = new HashSet<>(matchingPaths.size());
- 		for ( final String pathname : matchingPaths ) {
- 			paths.add(new Path(pathname));
- 		}
- 		return paths;
- 	}
++    private Date latestTimestamp;
++    private Collection<String> matchingPaths;
 +
- 	/**
- 	 * Sets the Collection containing the paths of all files in the HDFS Listing whose Modification Date was
- 	 * equal to {@link #getLatestTimestamp()}
- 	 * @param matchingPaths
- 	 */
- 	public void setMatchingPaths(Collection<String> matchingPaths) {
- 		this.matchingPaths = matchingPaths;
- 	}
++    /**
++     * @return the modification date of the newest file that was contained in the HDFS Listing
++     */
++    public Date getLatestTimestamp() {
++        return latestTimestamp;
++    }
++
++    /**
++     * Sets the timestamp of the modification date of the newest file that was contained in the HDFS Listing
++     *
++     * @param latestTimestamp the timestamp of the modification date of the newest file that was contained in the HDFS Listing
++     */
++    public void setLatestTimestamp(Date latestTimestamp) {
++        this.latestTimestamp = latestTimestamp;
++    }
++
++    /**
++     * @return a Collection containing the paths of all files in the HDFS Listing whose Modification date
++     * was equal to {@link #getLatestTimestamp()}
++     */
++    @XmlTransient
++    public Collection<String> getMatchingPaths() {
++        return matchingPaths;
++    }
++
++    /**
++     * @return a Collection of {@link Path} objects equivalent to those returned by {@link #getMatchingPaths()}
++     */
++    public Set<Path> toPaths() {
++        final Set<Path> paths = new HashSet<>(matchingPaths.size());
++        for ( final String pathname : matchingPaths ) {
++            paths.add(new Path(pathname));
++        }
++        return paths;
++    }
++
++    /**
++     * Sets the Collection containing the paths of all files in the HDFS Listing whose Modification Date was
++     * equal to {@link #getLatestTimestamp()}
++     * @param matchingPaths the paths that have last modified date matching the latest timestamp
++     */
++    public void setMatchingPaths(Collection<String> matchingPaths) {
++        this.matchingPaths = matchingPaths;
++    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
index ef0e590,0000000..229f26c
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/LongSerDe.java
@@@ -1,48 -1,0 +1,48 @@@
 +/*
 + * 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.hadoop.util;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.OutputStream;
 +
 +import org.apache.nifi.distributed.cache.client.Deserializer;
 +import org.apache.nifi.distributed.cache.client.Serializer;
 +import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
 +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
 +
 +public class LongSerDe implements Serializer<Long>, Deserializer<Long> {
 +
- 	@Override
- 	public Long deserialize(final byte[] input) throws DeserializationException, IOException {
- 		if ( input == null || input.length == 0 ) {
- 			return null;
- 		}
- 		
- 		final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(input));
- 		return dis.readLong();
- 	}
++    @Override
++    public Long deserialize(final byte[] input) throws DeserializationException, IOException {
++        if ( input == null || input.length == 0 ) {
++            return null;
++        }
 +
- 	@Override
- 	public void serialize(final Long value, final OutputStream out) throws SerializationException, IOException {
- 		final DataOutputStream dos = new DataOutputStream(out);
- 		dos.writeLong(value);
- 	}
++        final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(input));
++        return dis.readLong();
++    }
++
++    @Override
++    public void serialize(final Long value, final OutputStream out) throws SerializationException, IOException {
++        final DataOutputStream dos = new DataOutputStream(out);
++        dos.writeLong(value);
++    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
index 848831f,0000000..ca1c548
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/util/StringSerDe.java
@@@ -1,44 -1,0 +1,44 @@@
 +/*
 + * 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.hadoop.util;
 +
 +import java.io.IOException;
 +import java.io.OutputStream;
 +import java.nio.charset.StandardCharsets;
 +
 +import org.apache.nifi.distributed.cache.client.Deserializer;
 +import org.apache.nifi.distributed.cache.client.Serializer;
 +import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
 +import org.apache.nifi.distributed.cache.client.exception.SerializationException;
 +
 +public class StringSerDe implements Serializer<String>, Deserializer<String> {
 +
- 	@Override
- 	public String deserialize(final byte[] value) throws DeserializationException, IOException {
- 		if ( value == null ) {
- 			return null;
- 		}
- 		
- 		return new String(value, StandardCharsets.UTF_8);
- 	}
++    @Override
++    public String deserialize(final byte[] value) throws DeserializationException, IOException {
++        if ( value == null ) {
++            return null;
++        }
 +
- 	@Override
- 	public void serialize(final String value, final OutputStream out) throws SerializationException, IOException {
- 		out.write(value.getBytes(StandardCharsets.UTF_8));
- 	}
++        return new String(value, StandardCharsets.UTF_8);
++    }
++
++    @Override
++    public void serialize(final String value, final OutputStream out) throws SerializationException, IOException {
++        out.write(value.getBytes(StandardCharsets.UTF_8));
++    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
index 975dc63,d816e8c..ea3bb63
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-client-service-api/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClient.java
@@@ -83,31 -86,15 +86,34 @@@ public interface DistributedMapCacheCli
      <K> boolean containsKey(K key, Serializer<K> keySerializer) throws IOException;
  
      /**
 -     * @param <K> type of key
 -     * @param <V> type of value
 +     * Adds the specified key and value to the cache, overwriting any value that is
 +     * currently set.
-      * 
++     *
++     * @param <K> the key type
++     * @param <V> the value type
 +     * @param key The key to set
 +     * @param value The value to associate with the given Key
 +     * @param keySerializer the Serializer that will be used to serialize the key into bytes
 +     * @param valueSerializer the Serializer that will be used to serialize the value into bytes
-      * 
++     *
 +     * @throws IOException if unable to communicate with the remote instance
 +     * @throws NullPointerException if the key or either serializer is null
 +     */
 +    <K, V> void put(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException;
-     
++
 +    /**
 +     * Returns the value in the cache for the given key, if one exists;
 +     * otherwise returns <code>null</code>
 +     *
-      * @param <K>
-      * @param <V>
++     * @param <K> the key type
++     * @param <V> the value type
       * @param key the key to lookup in the map
-      * @param keySerializer
-      * @param valueDeserializer
+      * @param keySerializer key serializer
+      * @param valueDeserializer value serializer
       *
-      * @return
-      * @throws IOException
+      * @return the value in the cache for the given key, if one exists;
+      * otherwise returns <code>null</code>
+      * @throws IOException ex
       */
      <K, V> V get(K key, Serializer<K> keySerializer, Deserializer<V> valueDeserializer) throws IOException;
  

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
index 8903046,fad0adb..e9c6f1d
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java
@@@ -22,9 -22,12 +22,14 @@@ import java.nio.ByteBuffer
  public interface MapCache {
  
      MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException;
+ 
 +    MapPutResult put(ByteBuffer key, ByteBuffer value) throws IOException;
++
      boolean containsKey(ByteBuffer key) throws IOException;
+ 
      ByteBuffer get(ByteBuffer key) throws IOException;
+ 
      ByteBuffer remove(ByteBuffer key) throws IOException;
+ 
      void shutdown() throws IOException;
  }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
index cf8996c,943d6aa..13ed0df
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java
@@@ -55,70 -55,63 +55,70 @@@ public class MapCacheServer extends Abs
          final String action = dis.readUTF();
          try {
              switch (action) {
 -                case "close": {
 -                    return false;
 -                }
 -                case "putIfAbsent": {
 -                    final byte[] key = readValue(dis);
 -                    final byte[] value = readValue(dis);
 -                    final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
 -                    dos.writeBoolean(putResult.isSuccessful());
 -                    break;
 -                }
 -                case "containsKey": {
 -                    final byte[] key = readValue(dis);
 -                    final boolean contains = cache.containsKey(ByteBuffer.wrap(key));
 -                    dos.writeBoolean(contains);
 -                    break;
 -                }
 -                case "getAndPutIfAbsent": {
 -                    final byte[] key = readValue(dis);
 -                    final byte[] value = readValue(dis);
 -
 -                    final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
 -                    if (putResult.isSuccessful()) {
 -                        // Put was successful. There was no old value to get.
 -                        dos.writeInt(0);
 -                    } else {
 -                        // we didn't put. Write back the previous value
 -                        final byte[] byteArray = putResult.getExistingValue().array();
 -                        dos.writeInt(byteArray.length);
 -                        dos.write(byteArray);
 -                    }
 -
 -                    break;
 -                }
 -                case "get": {
 -                    final byte[] key = readValue(dis);
 -                    final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key));
 -                    if (existingValue == null) {
 -                        // there was no existing value; we did a "put".
 -                        dos.writeInt(0);
 -                    } else {
 -                        // a value already existed. we did not update the map
 -                        final byte[] byteArray = existingValue.array();
 -                        dos.writeInt(byteArray.length);
 -                        dos.write(byteArray);
 -                    }
 -
 -                    break;
 -                }
 -                case "remove": {
 -                    final byte[] key = readValue(dis);
 -                    final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null;
 -                    dos.writeBoolean(removed);
 -                    break;
 +            case "close": {
 +                return false;
 +            }
 +            case "putIfAbsent": {
 +                final byte[] key = readValue(dis);
 +                final byte[] value = readValue(dis);
 +                final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
 +                dos.writeBoolean(putResult.isSuccessful());
 +                break;
 +            }
 +            case "put": {
-             	final byte[] key = readValue(dis);
-             	final byte[] value = readValue(dis);
-             	cache.put(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
++                final byte[] key = readValue(dis);
++                final byte[] value = readValue(dis);
++                cache.put(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
 +                dos.writeBoolean(true);
-             	break;
++                break;
 +            }
 +            case "containsKey": {
 +                final byte[] key = readValue(dis);
 +                final boolean contains = cache.containsKey(ByteBuffer.wrap(key));
 +                dos.writeBoolean(contains);
 +                break;
 +            }
 +            case "getAndPutIfAbsent": {
 +                final byte[] key = readValue(dis);
 +                final byte[] value = readValue(dis);
 +
 +                final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
 +                if (putResult.isSuccessful()) {
 +                    // Put was successful. There was no old value to get.
 +                    dos.writeInt(0);
 +                } else {
 +                    // we didn't put. Write back the previous value
 +                    final byte[] byteArray = putResult.getExistingValue().array();
 +                    dos.writeInt(byteArray.length);
 +                    dos.write(byteArray);
                  }
 -                default: {
 -                    throw new IOException("Illegal Request");
 +
 +                break;
 +            }
 +            case "get": {
 +                final byte[] key = readValue(dis);
 +                final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key));
 +                if (existingValue == null) {
 +                    // there was no existing value; we did a "put".
 +                    dos.writeInt(0);
 +                } else {
 +                    // a value already existed. we did not update the map
 +                    final byte[] byteArray = existingValue.array();
 +                    dos.writeInt(byteArray.length);
 +                    dos.write(byteArray);
                  }
 +
 +                break;
 +            }
 +            case "remove": {
 +                final byte[] key = readValue(dis);
 +                final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null;
 +                dos.writeBoolean(removed);
 +                break;
 +            }
 +            default: {
 +                throw new IOException("Illegal Request");
 +            }
              }
          } finally {
              dos.flush();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
index 82b1787,e821fbf..663f441
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java
@@@ -75,33 -75,9 +75,33 @@@ public class PersistentMapCache impleme
                  wali.checkpoint();
              }
          }
-         
+ 
          return putResult;
      }
 +    
 +    @Override
 +    public MapPutResult put(final ByteBuffer key, final ByteBuffer value) throws IOException {
 +    	final MapPutResult putResult = wrapped.put(key, value);
 +        if ( putResult.isSuccessful() ) {
 +            // The put was successful.
 +            final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value);
 +            final List<MapWaliRecord> records = new ArrayList<>();
 +            records.add(record);
 +
 +            if ( putResult.getEvictedKey() != null ) {
 +                records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue()));
 +            }
 +            
 +            wali.update(Collections.singletonList(record), false);
 +            
 +            final long modCount = modifications.getAndIncrement();
 +            if ( modCount > 0 && modCount % 100000 == 0 ) {
 +                wali.checkpoint();
 +            }
 +        }
 +        
 +        return putResult;
 +    }
  
      @Override
      public boolean containsKey(final ByteBuffer key) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dc7f7a82/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
index d8f9c45,9e8bbd1..b167c62
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java
@@@ -105,29 -106,7 +106,29 @@@ public class SimpleMapCache implements 
              writeLock.unlock();
          }
      }
-     
+ 
 +
 +    @Override
 +    public MapPutResult put(final ByteBuffer key, final ByteBuffer value) {
 +        writeLock.lock();
 +        try {
-         	// evict if we need to in order to make room for a new entry.
++            // evict if we need to in order to make room for a new entry.
 +            final MapCacheRecord evicted = evict();
 +
 +            final MapCacheRecord record = new MapCacheRecord(key, value);
-         	final MapCacheRecord existing = cache.put(key, record);
-         	inverseCacheMap.put(record, key);
-         	
-         	final ByteBuffer existingValue = (existing == null) ? null : existing.getValue();
-         	final ByteBuffer evictedKey = (evicted == null) ? null : evicted.getKey();
-         	final ByteBuffer evictedValue = (evicted == null) ? null : evicted.getValue();
-         	
-         	return new MapPutResult(true, key, value, existingValue, evictedKey, evictedValue);
++            final MapCacheRecord existing = cache.put(key, record);
++            inverseCacheMap.put(record, key);
++
++            final ByteBuffer existingValue = (existing == null) ? null : existing.getValue();
++            final ByteBuffer evictedKey = (evicted == null) ? null : evicted.getKey();
++            final ByteBuffer evictedValue = (evicted == null) ? null : evicted.getValue();
++
++            return new MapPutResult(true, key, value, existingValue, evictedKey, evictedValue);
 +        } finally {
 +            writeLock.unlock();
 +        }
 +    }
-     
++
      @Override
      public boolean containsKey(final ByteBuffer key) {
          readLock.lock();