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/11/02 20:31:58 UTC

[01/50] [abbrv] nifi git commit: [PATCH] NIFI-997 Periodically attempt a kerberos relogin in AbstractHadoopProcessor

Repository: nifi
Updated Branches:
  refs/heads/NIFI-730 5a04021dd -> dbf0c7893


[PATCH] NIFI-997 Periodically attempt a kerberos relogin in AbstractHadoopProcessor

 - attempt a relogin based on an interval specified in the processor configuration
 - use hadoop's UserGroupInformation.checkTGTAndReloginFromKeytab to determine if a relogin is necessary based on the ticket and do so if needed
 - improve code readability with HdfsResources object in AbstractHadoopProcessor

Reviewed and Amended by Tony Kurc (tkurc@apache.org). This closes #97


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

Branch: refs/heads/NIFI-730
Commit: f2c4f2d2a15f23edd8933f6dbd10210de1700de3
Parents: d63cd6b
Author: ricky <ri...@cloudera.com>
Authored: Sun Oct 25 00:49:58 2015 -0400
Committer: Tony Kurc <tr...@gmail.com>
Committed: Sun Oct 25 00:49:58 2015 -0400

----------------------------------------------------------------------
 .../hadoop/AbstractHadoopProcessor.java         | 92 +++++++++++++++++---
 1 file changed, 79 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/f2c4f2d2/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 0102b1f..a67a9fd 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -25,6 +25,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import javax.net.SocketFactory;
@@ -50,9 +51,9 @@ import org.apache.nifi.components.Validator;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.Tuple;
 
 /**
  * This is a base class that is helpful when building processors interacting with HDFS.
@@ -132,15 +133,24 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             .description("Kerberos keytab associated with the principal. Requires nifi.kerberos.krb5.file to be set " + "in your nifi.properties").addValidator(Validator.VALID)
             .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).addValidator(KERBEROS_CONFIG_VALIDATOR).build();
 
+    private static final PropertyDescriptor KERBEROS_RELOGIN_PERIOD = new PropertyDescriptor.Builder().name("Kerberos Relogin Period").required(false)
+            .description("Period of time which should pass before attempting a kerberos relogin").defaultValue("4 hours")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
     protected static final List<PropertyDescriptor> properties;
 
     private static final Object RESOURCES_LOCK = new Object();
 
+    private long kerberosReloginThreshold;
+    private long lastKerberosReloginTime;
+
     static {
         List<PropertyDescriptor> props = new ArrayList<>();
         props.add(HADOOP_CONFIGURATION_RESOURCES);
         props.add(KERBEROS_PRINCIPAL);
         props.add(KERBEROS_KEYTAB);
+        props.add(KERBEROS_RELOGIN_PERIOD);
         properties = Collections.unmodifiableList(props);
         try {
             NIFI_PROPERTIES = NiFiProperties.getInstance();
@@ -154,12 +164,12 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     }
 
     // variables shared by all threads of this processor
-    // Hadoop Configuration and FileSystem
-    private final AtomicReference<Tuple<Configuration, FileSystem>> hdfsResources = new AtomicReference<>();
+    // Hadoop Configuration, Filesystem, and UserGroupInformation (optional)
+    private final AtomicReference<HdfsResources> hdfsResources = new AtomicReference<>();
 
     @Override
     protected void init(ProcessorInitializationContext context) {
-        hdfsResources.set(new Tuple<Configuration, FileSystem>(null, null));
+        hdfsResources.set(new HdfsResources(null, null, null));
     }
 
     @Override
@@ -173,8 +183,13 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     @OnScheduled
     public final void abstractOnScheduled(ProcessContext context) throws IOException {
         try {
-            Tuple<Configuration, FileSystem> resources = hdfsResources.get();
-            if (resources.getKey() == null || resources.getValue() == null) {
+            // This value will be null when called from ListHDFS, because it overrides all of the default
+            // properties this processor sets. TODO: re-work ListHDFS to utilize Kerberos
+            if (context.getProperty(KERBEROS_RELOGIN_PERIOD).getValue() != null) {
+                kerberosReloginThreshold = context.getProperty(KERBEROS_RELOGIN_PERIOD).asTimePeriod(TimeUnit.SECONDS);
+            }
+            HdfsResources resources = hdfsResources.get();
+            if (resources.getConfiguration() == null) {
                 String configResources = context.getProperty(HADOOP_CONFIGURATION_RESOURCES).getValue();
                 String dir = context.getProperty(DIRECTORY_PROP_NAME).getValue();
                 dir = dir == null ? "/" : dir;
@@ -183,14 +198,14 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             }
         } catch (IOException ex) {
             getLogger().error("HDFS Configuration error - {}", new Object[] { ex });
-            hdfsResources.set(new Tuple<Configuration, FileSystem>(null, null));
+            hdfsResources.set(new HdfsResources(null, null, null));
             throw ex;
         }
     }
 
     @OnStopped
     public final void abstractOnStopped() {
-        hdfsResources.set(new Tuple<Configuration, FileSystem>(null, null));
+        hdfsResources.set(new HdfsResources(null, null, null));
     }
 
     private static Configuration getConfigurationFromResources(String configResources) throws IOException {
@@ -224,7 +239,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     /*
      * Reset Hadoop Configuration and FileSystem based on the supplied configuration resources.
      */
-    Tuple<Configuration, FileSystem> resetHDFSResources(String configResources, String dir, ProcessContext context) throws IOException {
+    HdfsResources resetHDFSResources(String configResources, String dir, ProcessContext context) throws IOException {
         // org.apache.hadoop.conf.Configuration saves its current thread context class loader to use for threads that it creates
         // later to do I/O. We need this class loader to be the NarClassLoader instead of the magical
         // NarThreadContextClassLoader.
@@ -244,13 +259,15 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             // If kerberos is enabled, create the file system as the kerberos principal
             // -- use RESOURCE_LOCK to guarantee UserGroupInformation is accessed by only a single thread at at time
             FileSystem fs = null;
+            UserGroupInformation ugi = null;
             synchronized (RESOURCES_LOCK) {
                 if (config.get("hadoop.security.authentication").equalsIgnoreCase("kerberos")) {
                     String principal = context.getProperty(KERBEROS_PRINCIPAL).getValue();
                     String keyTab = context.getProperty(KERBEROS_KEYTAB).getValue();
                     UserGroupInformation.setConfiguration(config);
-                    UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keyTab);
+                    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keyTab);
                     fs = getFileSystemAsUser(config, ugi);
+                    lastKerberosReloginTime = System.currentTimeMillis() / 1000;
                 } else {
                     config.set("ipc.client.fallback-to-simple-auth-allowed", "true");
                     config.set("hadoop.security.authentication", "simple");
@@ -260,7 +277,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             config.set(disableCacheName, "true");
             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)), fs.getDefaultReplication(new Path(dir)), config.toString() });
-            return new Tuple<>(config, fs);
+            return new HdfsResources(config, fs, ugi);
 
         } finally {
             Thread.currentThread().setContextClassLoader(savedClassLoader);
@@ -392,10 +409,59 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     }
 
     protected Configuration getConfiguration() {
-        return hdfsResources.get().getKey();
+        return hdfsResources.get().getConfiguration();
     }
 
     protected FileSystem getFileSystem() {
-        return hdfsResources.get().getValue();
+        // if kerberos is enabled, check if the ticket should be renewed before returning the FS
+        if (hdfsResources.get().getUserGroupInformation() != null && isTicketOld()) {
+            tryKerberosRelogin(hdfsResources.get().getUserGroupInformation());
+        }
+        return hdfsResources.get().getFileSystem();
+    }
+
+    protected void tryKerberosRelogin(UserGroupInformation ugi) {
+        try {
+            getLogger().info("Kerberos ticket age exceeds threshold [{} seconds] " +
+                "attempting to renew ticket for user {}", new Object[]{
+              kerberosReloginThreshold, ugi.getUserName()});
+            ugi.checkTGTAndReloginFromKeytab();
+            lastKerberosReloginTime = System.currentTimeMillis() / 1000;
+            getLogger().info("Kerberos relogin successful or ticket still valid");
+        } catch (IOException e) {
+            // Most likely case of this happening is ticket is expired and error getting a new one,
+            // meaning dfs operations would fail
+            getLogger().error("Kerberos relogin failed", e);
+            throw new ProcessException("Unable to renew kerberos ticket", e);
+        }
+    }
+
+    protected boolean isTicketOld() {
+        return (System.currentTimeMillis() / 1000 - lastKerberosReloginTime) > kerberosReloginThreshold;
+    }
+
+
+    static protected class HdfsResources {
+        private final Configuration configuration;
+        private final FileSystem fileSystem;
+        private final UserGroupInformation userGroupInformation;
+
+        public HdfsResources(Configuration configuration, FileSystem fileSystem, UserGroupInformation userGroupInformation) {
+            this.configuration = configuration;
+            this.fileSystem = fileSystem;
+            this.userGroupInformation = userGroupInformation;
+        }
+
+        public Configuration getConfiguration() {
+            return configuration;
+        }
+
+        public FileSystem getFileSystem() {
+            return fileSystem;
+        }
+
+        public UserGroupInformation getUserGroupInformation() {
+            return userGroupInformation;
+        }
     }
 }


[42/50] [abbrv] nifi git commit: NIFI-1090: Fixed log message that was at info level but should have been debug level

Posted by ma...@apache.org.
NIFI-1090: Fixed log message that was at info level but should have been debug level


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

Branch: refs/heads/NIFI-730
Commit: ad849c77dff7b379116f4d57510c7b9136c7f4c0
Parents: b729bf4
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Nov 1 14:37:01 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Nov 1 14:37:01 2015 -0500

----------------------------------------------------------------------
 .../nifi/controller/repository/FileSystemRepository.java  | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/ad849c77/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 72a50ec..5baddbb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -1208,7 +1208,15 @@ public class FileSystemRepository implements ContentRepository {
         final long startNanos = System.nanoTime();
         final long toFree = minRequiredSpace - usableSpace;
         final BlockingQueue<ArchiveInfo> fileQueue = archivedFiles.get(containerName);
-        archiveExpirationLog.info("Currently {} bytes free for Container {}; requirement is {} byte free, so need to free {} bytes", usableSpace, containerName, minRequiredSpace, toFree);
+        if (archiveExpirationLog.isDebugEnabled()) {
+            if (toFree < 0) {
+                archiveExpirationLog.debug("Currently {} bytes free for Container {}; requirement is {} byte free, so no need to free space until an additional {} bytes are used",
+                    usableSpace, containerName, minRequiredSpace, Math.abs(toFree));
+            } else {
+                archiveExpirationLog.debug("Currently {} bytes free for Container {}; requirement is {} byte free, so need to free {} bytes",
+                    usableSpace, containerName, minRequiredSpace, toFree);
+            }
+        }
 
         ArchiveInfo toDelete;
         int deleteCount = 0;


[02/50] [abbrv] nifi git commit: NIFI-810: Merged master into branch

Posted by ma...@apache.org.
NIFI-810: Merged master into branch


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

Branch: refs/heads/NIFI-730
Commit: 0636f0e731cd28299edd3a6e9db90de5045ab662
Parents: 8e2308b d63cd6b
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 25 11:02:40 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:02:40 2015 -0400

----------------------------------------------------------------------
 .../src/main/asciidoc/administration-guide.adoc |   4 +-
 .../src/main/java/MyProcessor.java              |  11 +-
 .../nifi/processors/avro/ConvertAvroToJSON.java |  67 ++++-
 .../processors/avro/TestConvertAvroToJSON.java  |  47 ++-
 .../processors/aws/AbstractAWSProcessor.java    |   2 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java  |  98 ++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../processors/aws/s3/TestDeleteS3Object.java   | 141 +++++++++
 .../nifi/controller/FlowUnmarshaller.java       |  77 -----
 .../src/main/resources/FlowConfiguration.xsd    |   2 +-
 .../src/main/resources/bin/nifi.sh              |  96 +++---
 .../canvas/new-controller-service-dialog.jsp    |   1 -
 .../partials/canvas/new-processor-dialog.jsp    |   1 -
 .../canvas/new-reporting-task-dialog.jsp        |   1 -
 .../css/new-controller-service-dialog.css       |   9 -
 .../main/webapp/css/new-processor-dialog.css    |   9 -
 .../webapp/css/new-reporting-task-dialog.css    |   9 -
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |  60 ++--
 .../src/main/webapp/js/nf/canvas/nf-settings.js | 140 +++++----
 .../processors/kite/AbstractKiteProcessor.java  |  11 +-
 .../nifi/processors/kite/ConvertCSVToAvro.java  | 296 ++++++++++---------
 .../processors/kite/TestCSVToAvroProcessor.java |  39 +++
 .../nifi-standard-prioritizers/pom.xml          |   4 +
 .../PriorityAttributePrioritizer.java           |   7 +-
 .../PriorityAttributePrioritizerTest.java       |  17 +-
 .../nifi-standard-processors/pom.xml            |   9 +
 .../nifi/processors/standard/ExecuteSQL.java    |   9 +-
 .../nifi/processors/standard/InvokeHTTP.java    |   1 +
 .../nifi/processors/standard/ListenHTTP.java    | 105 ++++---
 .../standard/PutDistributedMapCache.java        | 252 ++++++++++++++++
 .../servlets/ContentAcknowledgmentServlet.java  |   3 +-
 .../standard/servlets/ListenHTTPServlet.java    |   8 +-
 .../processors/standard/util/JdbcCommon.java    |  70 ++++-
 .../org.apache.nifi.processor.Processor         |   1 +
 .../nifi/processors/standard/TestGetFile.java   |  21 +-
 .../standard/TestPutDistributedMapCache.java    | 277 +++++++++++++++++
 .../standard/util/TestJdbcCommon.java           |  42 +++
 .../standard/util/TestJdbcTypesDerby.java       | 133 +++++++++
 .../standard/util/TestJdbcTypesH2.java          | 149 ++++++++++
 pom.xml                                         |   2 +-
 40 files changed, 1725 insertions(+), 507 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0636f0e7/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java
----------------------------------------------------------------------
diff --cc nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java
index b214427,f0ba71a..f0f1630
--- a/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java
+++ b/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java
@@@ -35,7 -36,8 +38,7 @@@ import org.apache.nifi.annotation.behav
  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.Tags;
+ import org.apache.nifi.components.PropertyDescriptor;
  import org.apache.nifi.flowfile.FlowFile;
  import org.apache.nifi.flowfile.attributes.CoreAttributes;
  import org.apache.nifi.processor.AbstractProcessor;
@@@ -47,8 -50,7 +51,7 @@@ import org.apache.nifi.processor.io.Str
  
  @SideEffectFree
  @SupportsBatching
- @Tags({ "json", "avro", "binary" })
 -@Tags({"json", "avro", "binary"})
 +@InputRequirement(Requirement.INPUT_REQUIRED)
  @CapabilityDescription("Converts a Binary Avro record into a JSON object. This processor provides a direct mapping of an Avro field to a JSON field, such "
      + "that the resulting JSON will have the same hierarchical structure as the Avro document. Note that the Avro schema information will be lost, as this "
      + "is not a translation from binary Avro to JSON formatted Avro. The output JSON is encoded the UTF-8 encoding. If an incoming FlowFile contains a stream of "

http://git-wip-us.apache.org/repos/asf/nifi/blob/0636f0e7/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
----------------------------------------------------------------------
diff --cc nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
index 6f126aa,ea84daa..43b33ff
--- a/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
+++ b/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java
@@@ -30,8 -30,7 +30,9 @@@ import org.apache.avro.Schema
  import org.apache.avro.file.CodecFactory;
  import org.apache.avro.file.DataFileWriter;
  import org.apache.avro.generic.GenericData.Record;
+ import org.apache.commons.lang3.StringEscapeUtils;
 +import org.apache.nifi.annotation.behavior.InputRequirement;
 +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
  import org.apache.nifi.annotation.documentation.CapabilityDescription;
  import org.apache.nifi.annotation.documentation.Tags;
  import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@@ -68,114 -66,118 +69,108 @@@ public class ConvertCSVToAvro extends A
  
      private static final Validator CHAR_VALIDATOR = new Validator() {
          @Override
--        public ValidationResult validate(String subject, String input,
--                ValidationContext context) {
++        public ValidationResult validate(String subject, String input, ValidationContext context) {
+             // Allows special, escaped characters as input, which is then unescaped and converted to a single character.
+             // Examples for special characters: \t (or \u0009), \f.
+             input = unescapeString(input);
+ 
              return new ValidationResult.Builder()
--                    .subject(subject)
--                    .input(input)
-                     .explanation("Only single characters are supported")
-                     .valid(input.length() == 1)
 -                    .explanation("Only non-null single characters are supported")
 -                    .valid(input.length() == 1 && input.charAt(0) != 0)
--                    .build();
++                .subject(subject)
++                .input(input)
++                .explanation("Only non-null single characters are supported")
++                .valid(input.length() == 1 && input.charAt(0) != 0)
++                .build();
          }
      };
  
      private static final Relationship SUCCESS = new Relationship.Builder()
--            .name("success")
--            .description("Avro content that was converted successfully from CSV")
--            .build();
++        .name("success")
++        .description("Avro content that was converted successfully from CSV")
++        .build();
  
      private static final Relationship FAILURE = new Relationship.Builder()
--            .name("failure")
--            .description("CSV content that could not be processed")
--            .build();
++        .name("failure")
++        .description("CSV content that could not be processed")
++        .build();
  
      private static final Relationship INCOMPATIBLE = new Relationship.Builder()
--            .name("incompatible")
--            .description("CSV content that could not be converted")
--            .build();
++        .name("incompatible")
++        .description("CSV content that could not be converted")
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor SCHEMA
--            = new PropertyDescriptor.Builder()
--            .name("Record schema")
--            .description("Outgoing Avro schema for each record created from a CSV row")
--            .addValidator(SCHEMA_VALIDATOR)
--            .expressionLanguageSupported(true)
--            .required(true)
--            .build();
++    static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder()
++        .name("Record schema")
++        .description("Outgoing Avro schema for each record created from a CSV row")
++        .addValidator(SCHEMA_VALIDATOR)
++        .expressionLanguageSupported(true)
++        .required(true)
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor CHARSET
--            = new PropertyDescriptor.Builder()
--            .name("CSV charset")
--            .description("Character set for CSV files")
--            .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
--            .defaultValue(DEFAULTS.charset)
--            .build();
++    static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
++        .name("CSV charset")
++        .description("Character set for CSV files")
++        .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
++        .defaultValue(DEFAULTS.charset)
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor DELIMITER
--            = new PropertyDescriptor.Builder()
--            .name("CSV delimiter")
--            .description("Delimiter character for CSV records")
--            .addValidator(CHAR_VALIDATOR)
--            .defaultValue(DEFAULTS.delimiter)
--            .build();
++    static final PropertyDescriptor DELIMITER = new PropertyDescriptor.Builder()
++        .name("CSV delimiter")
++        .description("Delimiter character for CSV records")
++        .addValidator(CHAR_VALIDATOR)
++        .defaultValue(DEFAULTS.delimiter)
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor QUOTE
--            = new PropertyDescriptor.Builder()
--            .name("CSV quote character")
--            .description("Quote character for CSV values")
--            .addValidator(CHAR_VALIDATOR)
--            .defaultValue(DEFAULTS.quote)
--            .build();
++    static final PropertyDescriptor QUOTE = new PropertyDescriptor.Builder()
++        .name("CSV quote character")
++        .description("Quote character for CSV values")
++        .addValidator(CHAR_VALIDATOR)
++        .defaultValue(DEFAULTS.quote)
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor ESCAPE
--            = new PropertyDescriptor.Builder()
--            .name("CSV escape character")
--            .description("Escape character for CSV values")
--            .addValidator(CHAR_VALIDATOR)
--            .defaultValue(DEFAULTS.escape)
--            .build();
++    static final PropertyDescriptor ESCAPE = new PropertyDescriptor.Builder()
++        .name("CSV escape character")
++        .description("Escape character for CSV values")
++        .addValidator(CHAR_VALIDATOR)
++        .defaultValue(DEFAULTS.escape)
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor HAS_HEADER
--            = new PropertyDescriptor.Builder()
--            .name("Use CSV header line")
--            .description("Whether to use the first line as a header")
--            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
--            .defaultValue(String.valueOf(DEFAULTS.useHeader))
--            .build();
++    static final PropertyDescriptor HAS_HEADER = new PropertyDescriptor.Builder()
++        .name("Use CSV header line")
++        .description("Whether to use the first line as a header")
++        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
++        .defaultValue(String.valueOf(DEFAULTS.useHeader))
++        .build();
  
      @VisibleForTesting
--    static final PropertyDescriptor LINES_TO_SKIP
--            = new PropertyDescriptor.Builder()
--            .name("Lines to skip")
--            .description("Number of lines to skip before reading header or data")
--            .addValidator(createLongValidator(0L, Integer.MAX_VALUE, true))
--            .defaultValue(String.valueOf(DEFAULTS.linesToSkip))
--            .build();
--
--    private static final List<PropertyDescriptor> PROPERTIES
--            = ImmutableList.<PropertyDescriptor>builder()
--            .addAll(AbstractKiteProcessor.getProperties())
--            .add(SCHEMA)
--            .add(CHARSET)
--            .add(DELIMITER)
--            .add(QUOTE)
--            .add(ESCAPE)
--            .add(HAS_HEADER)
--            .add(LINES_TO_SKIP)
--            .build();
--
--    private static final Set<Relationship> RELATIONSHIPS
--            = ImmutableSet.<Relationship>builder()
--            .add(SUCCESS)
--            .add(FAILURE)
--            .add(INCOMPATIBLE)
--            .build();
++    static final PropertyDescriptor LINES_TO_SKIP = new PropertyDescriptor.Builder()
++        .name("Lines to skip")
++        .description("Number of lines to skip before reading header or data")
++        .addValidator(createLongValidator(0L, Integer.MAX_VALUE, true))
++        .defaultValue(String.valueOf(DEFAULTS.linesToSkip))
++        .build();
++
++    private static final List<PropertyDescriptor> PROPERTIES = ImmutableList.<PropertyDescriptor> builder()
++        .addAll(AbstractKiteProcessor.getProperties())
++        .add(SCHEMA)
++        .add(CHARSET)
++        .add(DELIMITER)
++        .add(QUOTE)
++        .add(ESCAPE)
++        .add(HAS_HEADER)
++        .add(LINES_TO_SKIP)
++        .build();
++
++    private static final Set<Relationship> RELATIONSHIPS = ImmutableSet.<Relationship> builder()
++        .add(SUCCESS)
++        .add(FAILURE)
++        .add(INCOMPATIBLE)
++        .build();
  
      // Immutable configuration
      @VisibleForTesting
@@@ -196,26 -198,26 +191,26 @@@
          super.setDefaultConfiguration(context);
  
          this.props = new CSVProperties.Builder()
--                .charset(context.getProperty(CHARSET).getValue())
--                .delimiter(context.getProperty(DELIMITER).getValue())
--                .quote(context.getProperty(QUOTE).getValue())
--                .escape(context.getProperty(ESCAPE).getValue())
--                .hasHeader(context.getProperty(HAS_HEADER).asBoolean())
--                .linesToSkip(context.getProperty(LINES_TO_SKIP).asInteger())
--                .build();
++            .charset(context.getProperty(CHARSET).getValue())
++            .delimiter(context.getProperty(DELIMITER).getValue())
++            .quote(context.getProperty(QUOTE).getValue())
++            .escape(context.getProperty(ESCAPE).getValue())
++            .hasHeader(context.getProperty(HAS_HEADER).asBoolean())
++            .linesToSkip(context.getProperty(LINES_TO_SKIP).asInteger())
++            .build();
      }
  
      @Override
      public void onTrigger(ProcessContext context, final ProcessSession session)
--            throws ProcessException {
++        throws ProcessException {
          FlowFile incomingCSV = session.get();
          if (incomingCSV == null) {
              return;
          }
  
          String schemaProperty = context.getProperty(SCHEMA)
--                .evaluateAttributeExpressions(incomingCSV)
--                .getValue();
++            .evaluateAttributeExpressions(incomingCSV)
++            .getValue();
          final Schema schema;
          try {
              schema = getSchema(schemaProperty, DefaultConfiguration.get());
@@@ -225,78 -227,85 +220,87 @@@
              return;
          }
  
--        final DataFileWriter<Record> writer = new DataFileWriter<>(
--                AvroUtil.newDatumWriter(schema, Record.class));
--        writer.setCodec(CodecFactory.snappyCodec());
++        try (final DataFileWriter<Record> writer = new DataFileWriter<>(AvroUtil.newDatumWriter(schema, Record.class))) {
++            writer.setCodec(CodecFactory.snappyCodec());
  
--        try {
--            final LongHolder written = new LongHolder(0L);
--            final FailureTracker failures = new FailureTracker();
--
--            FlowFile badRecords = session.clone(incomingCSV);
--            FlowFile outgoingAvro = session.write(incomingCSV, new StreamCallback() {
--                @Override
--                public void process(InputStream in, OutputStream out) throws IOException {
--                    try (CSVFileReader<Record> reader = new CSVFileReader<>(
++            try {
++                final LongHolder written = new LongHolder(0L);
++                final FailureTracker failures = new FailureTracker();
++
++                FlowFile badRecords = session.clone(incomingCSV);
++                FlowFile outgoingAvro = session.write(incomingCSV, new StreamCallback() {
++                    @Override
++                    public void process(InputStream in, OutputStream out) throws IOException {
++                        try (CSVFileReader<Record> reader = new CSVFileReader<>(
                              in, props, schema, Record.class)) {
--                        reader.initialize();
--                        try (DataFileWriter<Record> w = writer.create(schema, out)) {
--                            while (reader.hasNext()) {
--                                try {
--                                    Record record = reader.next();
--                                    w.append(record);
--                                    written.incrementAndGet();
--                                } catch (DatasetRecordException e) {
--                                    failures.add(e);
++                            reader.initialize();
++                            try (DataFileWriter<Record> w = writer.create(schema, out)) {
++                                while (reader.hasNext()) {
++                                    try {
++                                        Record record = reader.next();
++                                        w.append(record);
++                                        written.incrementAndGet();
++                                    } catch (DatasetRecordException e) {
++                                        failures.add(e);
++                                    }
                                  }
                              }
                          }
                      }
--                }
--            });
++                });
  
--            long errors = failures.count();
++                long errors = failures.count();
  
--            session.adjustCounter("Converted records", written.get(),
++                session.adjustCounter("Converted records", written.get(),
                      false /* update only if file transfer is successful */);
--            session.adjustCounter("Conversion errors", errors,
++                session.adjustCounter("Conversion errors", errors,
                      false /* update only if file transfer is successful */);
  
--            if (written.get() > 0L) {
--                session.transfer(outgoingAvro, SUCCESS);
++                if (written.get() > 0L) {
++                    session.transfer(outgoingAvro, SUCCESS);
  
--                if (errors > 0L) {
--                    getLogger().warn("Failed to convert {}/{} records from CSV to Avro",
--                            new Object[] { errors, errors + written.get() });
--                    badRecords = session.putAttribute(
++                    if (errors > 0L) {
++                        getLogger().warn("Failed to convert {}/{} records from CSV to Avro",
++                            new Object[] {errors, errors + written.get()});
++                        badRecords = session.putAttribute(
                              badRecords, "errors", failures.summary());
--                    session.transfer(badRecords, INCOMPATIBLE);
--                } else {
--                    session.remove(badRecords);
--                }
++                        session.transfer(badRecords, INCOMPATIBLE);
++                    } else {
++                        session.remove(badRecords);
++                    }
  
--            } else {
--                session.remove(outgoingAvro);
++                } else {
++                    session.remove(outgoingAvro);
  
--                if (errors > 0L) {
--                    getLogger().warn("Failed to convert {}/{} records from CSV to Avro",
--                            new Object[] { errors, errors });
--                    badRecords = session.putAttribute(
++                    if (errors > 0L) {
++                        getLogger().warn("Failed to convert {}/{} records from CSV to Avro",
++                            new Object[] {errors, errors});
++                        badRecords = session.putAttribute(
                              badRecords, "errors", failures.summary());
--                } else {
--                    badRecords = session.putAttribute(
++                    } else {
++                        badRecords = session.putAttribute(
                              badRecords, "errors", "No incoming records");
++                    }
++
++                    session.transfer(badRecords, FAILURE);
                  }
  
--                session.transfer(badRecords, FAILURE);
++            } catch (ProcessException | DatasetIOException e) {
++                getLogger().error("Failed reading or writing", e);
++                session.transfer(incomingCSV, FAILURE);
++            } catch (DatasetException e) {
++                getLogger().error("Failed to read FlowFile", e);
++                session.transfer(incomingCSV, FAILURE);
              }
 -
 -        } catch (ProcessException | DatasetIOException e) {
 -            getLogger().error("Failed reading or writing", e);
 -            session.transfer(incomingCSV, FAILURE);
 -        } catch (DatasetException e) {
 -            getLogger().error("Failed to read FlowFile", e);
 -            session.transfer(incomingCSV, FAILURE);
++        } catch (final IOException ioe) {
++            throw new RuntimeException("Unable to close Avro Writer", ioe);
+         }
+     }
  
-         } catch (ProcessException | DatasetIOException e) {
-             getLogger().error("Failed reading or writing", e);
-             session.transfer(incomingCSV, FAILURE);
-         } catch (DatasetException e) {
-             getLogger().error("Failed to read FlowFile", e);
-             session.transfer(incomingCSV, FAILURE);
+     private static String unescapeString(String input) {
+         if (input.length() > 1) {
+             input = StringEscapeUtils.unescapeJava(input);
          }
+         return input;
      }
  }

http://git-wip-us.apache.org/repos/asf/nifi/blob/0636f0e7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/0636f0e7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/0636f0e7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --cc nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 258e122,9ad1703..88b6666
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@@ -63,9 -61,8 +63,9 @@@ import org.eclipse.jetty.servlet.Servle
  import org.eclipse.jetty.util.ssl.SslContextFactory;
  import org.eclipse.jetty.util.thread.QueuedThreadPool;
  
 +@InputRequirement(Requirement.INPUT_FORBIDDEN)
  @Tags({"ingest", "http", "https", "rest", "listen"})
- @CapabilityDescription("Starts an HTTP Server that is used to receive FlowFiles from remote sources. The URL of the Service will be http://{hostname}:{port}/contentListener")
+ @CapabilityDescription("Starts an HTTP Server that is used to receive FlowFiles from remote sources. The default URI of the Service will be http://{hostname}:{port}/contentListener")
  public class ListenHTTP extends AbstractSessionFactoryProcessor {
  
      private Set<Relationship> relationships;


[12/50] [abbrv] nifi git commit: NIFI-1070: Added detailed debug-level logging about how FileSystemRepository is choosing to expire archived data

Posted by ma...@apache.org.
NIFI-1070: Added detailed debug-level logging about how FileSystemRepository is choosing to expire archived data


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

Branch: refs/heads/NIFI-730
Commit: aec32a277c3f2b707edd20a9eff9c4984f4f28fa
Parents: f8c3377
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Oct 26 14:36:03 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 14:36:03 2015 -0400

----------------------------------------------------------------------
 .../repository/FileSystemRepository.java        | 26 ++++++++++++++------
 1 file changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/aec32a27/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 724e26e..72a50ec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -87,6 +87,8 @@ public class FileSystemRepository implements ContentRepository {
     public static final Pattern MAX_ARCHIVE_SIZE_PATTERN = Pattern.compile("\\d{1,2}%");
     private static final Logger LOG = LoggerFactory.getLogger(FileSystemRepository.class);
 
+    private final Logger archiveExpirationLog = LoggerFactory.getLogger(FileSystemRepository.class.getName() + ".archive.expiration");
+
     private final Map<String, Path> containers;
     private final List<String> containerNames;
     private final AtomicLong index;
@@ -151,7 +153,7 @@ public class FileSystemRepository implements ContentRepository {
 
             if (maxArchiveSize == null) {
                 throw new RuntimeException("No value specified for property '"
-                        + NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE + "' but archiving is enabled. You must configure the max disk usage in order to enable archiving.");
+                    + NiFiProperties.CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE + "' but archiving is enabled. You must configure the max disk usage in order to enable archiving.");
             }
 
             if (!MAX_ARCHIVE_SIZE_PATTERN.matcher(maxArchiveSize.trim()).matches()) {
@@ -185,7 +187,7 @@ public class FileSystemRepository implements ContentRepository {
                 final long maxArchiveBytes = (long) (capacity * (1D - (maxArchiveRatio - 0.02)));
                 minUsableContainerBytesForArchive.put(container.getKey(), Long.valueOf(maxArchiveBytes));
                 LOG.info("Maximum Threshold for Container {} set to {} bytes; if volume exceeds this size, archived data will be deleted until it no longer exceeds this size",
-                        containerName, maxArchiveBytes);
+                    containerName, maxArchiveBytes);
 
                 final long backPressureBytes = (long) (Files.getFileStore(container.getValue()).getTotalSpace() * archiveBackPressureRatio);
                 final ContainerState containerState = new ContainerState(containerName, true, backPressureBytes, capacity);
@@ -620,7 +622,7 @@ public class FileSystemRepository implements ContentRepository {
 
         final File file = path.toFile();
         if (!file.delete() && file.exists()) {
-            LOG.warn("Unable to delete {} at path {}", new Object[]{claim, path});
+            LOG.warn("Unable to delete {} at path {}", new Object[] {claim, path});
             return false;
         }
 
@@ -1051,7 +1053,7 @@ public class FileSystemRepository implements ContentRepository {
                                     break;
                                 } else {
                                     LOG.warn("Failed to clean up {} because old claims aren't being cleaned up fast enough. "
-                                            + "This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim);
+                                        + "This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim);
                                 }
                             }
                         } catch (final InterruptedException ie) {
@@ -1187,6 +1189,7 @@ public class FileSystemRepository implements ContentRepository {
     }
 
     private long destroyExpiredArchives(final String containerName, final Path container) throws IOException {
+        archiveExpirationLog.debug("Destroying Expired Archives for Container {}", containerName);
         final List<ArchiveInfo> notYetExceedingThreshold = new ArrayList<>();
         final long removalTimeThreshold = System.currentTimeMillis() - maxArchiveMillis;
         long oldestArchiveDateFound = System.currentTimeMillis();
@@ -1194,6 +1197,7 @@ public class FileSystemRepository implements ContentRepository {
         // determine how much space we must have in order to stop deleting old data
         final Long minRequiredSpace = minUsableContainerBytesForArchive.get(containerName);
         if (minRequiredSpace == null) {
+            archiveExpirationLog.debug("Could not determine minimum required space so will not destroy any archived data");
             return -1L;
         }
 
@@ -1204,6 +1208,7 @@ public class FileSystemRepository implements ContentRepository {
         final long startNanos = System.nanoTime();
         final long toFree = minRequiredSpace - usableSpace;
         final BlockingQueue<ArchiveInfo> fileQueue = archivedFiles.get(containerName);
+        archiveExpirationLog.info("Currently {} bytes free for Container {}; requirement is {} byte free, so need to free {} bytes", usableSpace, containerName, minRequiredSpace, toFree);
 
         ArchiveInfo toDelete;
         int deleteCount = 0;
@@ -1217,7 +1222,7 @@ public class FileSystemRepository implements ContentRepository {
                 // In order to accomplish this, we just peek at the head and check if it should be deleted.
                 // If so, then we call poll() to remove it
                 if (freed < toFree || getLastModTime(toDelete.toPath()) < removalTimeThreshold) {
-                    toDelete = fileQueue.poll();   // remove the head of the queue, which is already stored in 'toDelete'
+                    toDelete = fileQueue.poll(); // remove the head of the queue, which is already stored in 'toDelete'
                     Files.deleteIfExists(toDelete.toPath());
                     containerState.decrementArchiveCount();
                     LOG.debug("Deleted archived ContentClaim with ID {} from Container {} because the archival size was exceeding the max configured size", toDelete.getName(), containerName);
@@ -1229,9 +1234,12 @@ public class FileSystemRepository implements ContentRepository {
                 if (freed >= toFree) {
                     // If the last mod time indicates that it should be removed, just continue loop.
                     if (deleteBasedOnTimestamp(fileQueue, removalTimeThreshold)) {
+                        archiveExpirationLog.debug("Freed enough space ({} bytes freed, needed to free {} bytes) but will continue to expire data based on timestamp", freed, toFree);
                         continue;
                     }
 
+                    archiveExpirationLog.debug("Freed enough space ({} bytes freed, needed to free {} bytes). Finished expiring data", freed, toFree);
+
                     final ArchiveInfo archiveInfo = fileQueue.peek();
                     final long oldestArchiveDate = archiveInfo == null ? System.currentTimeMillis() : getLastModTime(archiveInfo.toPath());
 
@@ -1256,6 +1264,7 @@ public class FileSystemRepository implements ContentRepository {
         }
 
         // Go through each container and grab the archived data into a List
+        archiveExpirationLog.debug("Searching for more archived data to expire");
         final StopWatch stopWatch = new StopWatch(true);
         for (int i = 0; i < SECTIONS_PER_CONTAINER; i++) {
             final Path sectionContainer = container.resolve(String.valueOf(i));
@@ -1278,7 +1287,7 @@ public class FileSystemRepository implements ContentRepository {
                                 Files.deleteIfExists(file);
                                 containerState.decrementArchiveCount();
                                 LOG.debug("Deleted archived ContentClaim with ID {} from Container {} because it was older than the configured max archival duration",
-                                        file.toFile().getName(), containerName);
+                                    file.toFile().getName(), containerName);
                             } catch (final IOException ioe) {
                                 LOG.warn("Failed to remove archived ContentClaim with ID {} from Container {} due to {}", file.toFile().getName(), containerName, ioe.toString());
                                 if (LOG.isDebugEnabled()) {
@@ -1312,6 +1321,7 @@ public class FileSystemRepository implements ContentRepository {
         final long sortRemainingMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS) - deleteExpiredMillis;
 
         // Delete the oldest data
+        archiveExpirationLog.debug("Deleting data based on timestamp");
         final Iterator<ArchiveInfo> itr = notYetExceedingThreshold.iterator();
         int counter = 0;
         while (itr.hasNext()) {
@@ -1325,7 +1335,7 @@ public class FileSystemRepository implements ContentRepository {
 
                 // Check if we've freed enough space every 25 files that we destroy
                 if (++counter % 25 == 0) {
-                    if (getContainerUsableSpace(containerName) > minRequiredSpace) {  // check if we can stop now
+                    if (getContainerUsableSpace(containerName) > minRequiredSpace) { // check if we can stop now
                         LOG.debug("Finished cleaning up archive for Container {}", containerName);
                         break;
                     }
@@ -1360,7 +1370,7 @@ public class FileSystemRepository implements ContentRepository {
 
         final long cleanupMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS) - deleteOldestMillis - sortRemainingMillis - deleteExpiredMillis;
         LOG.debug("Oldest Archive Date for Container {} is {}; delete expired = {} ms, sort remaining = {} ms, delete oldest = {} ms, cleanup = {} ms",
-                containerName, new Date(oldestContainerArchive), deleteExpiredMillis, sortRemainingMillis, deleteOldestMillis, cleanupMillis);
+            containerName, new Date(oldestContainerArchive), deleteExpiredMillis, sortRemainingMillis, deleteOldestMillis, cleanupMillis);
         return oldestContainerArchive;
     }
 


[40/50] [abbrv] nifi git commit: Merge branch 'NIFI-1088'

Posted by ma...@apache.org.
Merge branch 'NIFI-1088'


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

Branch: refs/heads/NIFI-730
Commit: 6e193dffa349b1f7627441489b8208666bea684a
Parents: 9c148da 9515b74
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Nov 1 14:16:35 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Nov 1 14:16:35 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/processors/kafka/PutKafka.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[33/50] [abbrv] nifi git commit: NIFI-1079 Replacing Name for NULL_VALUE_FOR_EMPTY_STRING as I had accidentally removed it.

Posted by ma...@apache.org.
NIFI-1079 Replacing Name for NULL_VALUE_FOR_EMPTY_STRING as I had accidentally removed it.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/NIFI-730
Commit: eb389cf84cd5f610b5ea2235a4eb78de47f6417e
Parents: aef0d8f
Author: Jeremy Dyer <jd...@gmail.com>
Authored: Fri Oct 30 14:15:59 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Oct 30 14:30:56 2015 -0400

----------------------------------------------------------------------
 .../org/apache/nifi/processors/standard/AttributesToJSON.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/eb389cf8/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
index 89bb0b6..0ab3ca7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
@@ -99,9 +99,9 @@ public class AttributesToJSON extends AbstractProcessor {
             .build();
 
     public static final PropertyDescriptor NULL_VALUE_FOR_EMPTY_STRING = new PropertyDescriptor.Builder()
-            .name(("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty " +
-                    "string will be placed in the JSON"))
-            .description("")
+            .name(("Null Value"))
+            .description("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty " +
+                    "string will be placed in the JSON")
             .required(true)
             .allowableValues("true", "false")
             .defaultValue("false")


[20/50] [abbrv] nifi git commit: Merge branch 'NIFI-447'

Posted by ma...@apache.org.
Merge branch 'NIFI-447'


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

Branch: refs/heads/NIFI-730
Commit: af19053a7f7e23f40530561530cfc56c0ad961d4
Parents: b885f95 07238c7
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Oct 27 10:42:46 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Oct 27 10:42:46 2015 -0400

----------------------------------------------------------------------
 .../nifi-standard-processors/pom.xml            |   5 +-
 .../nifi/processors/standard/ReplaceText.java   | 431 +++++++++----
 .../processors/standard/TestReplaceText.java    | 635 ++++++++++++++++++-
 .../standard/TestReplaceTextLineByLine.java     | 336 ----------
 .../AppendLineByLineTest.txt                    |  11 +
 .../PrependLineByLineTest.txt                   |  11 +
 6 files changed, 953 insertions(+), 476 deletions(-)
----------------------------------------------------------------------



[38/50] [abbrv] nifi git commit: NIFI-869: Fixed checkstyle violations

Posted by ma...@apache.org.
NIFI-869: Fixed checkstyle violations


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

Branch: refs/heads/NIFI-730
Commit: 9c148da1c28fa641b94cd7e8f1f9f07c63f2896a
Parents: ba72452
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Oct 30 16:09:50 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Oct 30 16:09:50 2015 -0400

----------------------------------------------------------------------
 .../nifi/processor/TestSimpleProcessLogger.java | 113 ++++++++++---------
 1 file changed, 57 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/9c148da1/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
index 2876abb..add54f3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
@@ -18,12 +18,12 @@ package org.apache.nifi.processor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.argThat;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.argThat;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.times;
 
 import java.lang.reflect.Field;
 
@@ -35,66 +35,67 @@ import org.mockito.internal.matchers.VarargMatcher;
 import org.slf4j.Logger;
 
 public class TestSimpleProcessLogger {
-	private final Exception e = new RuntimeException("intentional");
+    private final Exception e = new RuntimeException("intentional");
+
+    private ReportingTask task;
 
-	private  ReportingTask task;
+    private SimpleProcessLogger componentLog;
 
-	private SimpleProcessLogger componentLog;
+    private Logger logger;
 
-	private Logger logger;
+    @Before
+    public void before() {
+        task = mock(ReportingTask.class);
+        when(task.getIdentifier()).thenReturn("foo");
+        when(task.toString()).thenReturn("MyTask");
+        componentLog = new SimpleProcessLogger(task.getIdentifier(), task);
+        try {
+            Field loggerField = componentLog.getClass().getDeclaredField("logger");
+            loggerField.setAccessible(true);
+            logger = mock(Logger.class);
+            loggerField.set(componentLog, logger);
+        } catch (Exception e) {
+            e.printStackTrace();
+            fail(e.getMessage());
+        }
+    }
 
-	@Before
-	public void before(){
-		task = mock(ReportingTask.class);
-		when(task.getIdentifier()).thenReturn("foo");
-		when(task.toString()).thenReturn("MyTask");
-		componentLog = new SimpleProcessLogger(task.getIdentifier(), task);
-		try {
-			Field loggerField = componentLog.getClass().getDeclaredField("logger");
-			loggerField.setAccessible(true);
-			logger = mock(Logger.class);
-			loggerField.set(componentLog, logger);
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
+    @Test
+    public void validateDelegateLoggerReceivesThrowableToStringOnError() {
+        componentLog.error("Hello {}", e);
+        verify(logger, times(1)).error(anyString(), argThat(new MyVarargMatcher()));
+    }
 
-	@Test
-	public void validateDelegateLoggerReceivesThrowableToStringOnError() {
-		componentLog.error("Hello {}", e);
-		verify(logger, times(1)).error(anyString(), argThat(new MyVarargMatcher()));
-	}
+    @Test
+    public void validateDelegateLoggerReceivesThrowableToStringOnInfo() {
+        componentLog.info("Hello {}", e);
+        verify(logger, times(1)).info(anyString(), argThat(new MyVarargMatcher()));
+    }
 
-	@Test
-	public void validateDelegateLoggerReceivesThrowableToStringOnInfo() {
-		componentLog.info("Hello {}", e);
-		verify(logger, times(1)).info(anyString(), argThat(new MyVarargMatcher()));
-	}
+    @Test
+    public void validateDelegateLoggerReceivesThrowableToStringOnTrace() {
+        componentLog.trace("Hello {}", e);
+        verify(logger, times(1)).trace(anyString(), argThat(new MyVarargMatcher()));
+    }
 
-	@Test
-	public void validateDelegateLoggerReceivesThrowableToStringOnTrace() {
-		componentLog.trace("Hello {}", e);
-		verify(logger, times(1)).trace(anyString(), argThat(new MyVarargMatcher()));
-	}
+    @Test
+    public void validateDelegateLoggerReceivesThrowableToStringOnWarn() {
+        componentLog.warn("Hello {}", e);
+        verify(logger, times(1)).warn(anyString(), argThat(new MyVarargMatcher()));
+    }
 
-	@Test
-	public void validateDelegateLoggerReceivesThrowableToStringOnWarn() {
-		componentLog.warn("Hello {}", e);
-		verify(logger, times(1)).warn(anyString(), argThat(new MyVarargMatcher()));
-	}
+    /**
+     *
+     */
+    private class MyVarargMatcher extends ArgumentMatcher<Object[]>implements VarargMatcher {
+        private static final long serialVersionUID = 1L;
 
-	/**
-	 *
-	 */
-	private class MyVarargMatcher extends ArgumentMatcher<Object[]> implements VarargMatcher {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public boolean matches(Object argument) {
-			Object[] args = (Object[]) argument;
-			assertEquals(task, args[0]);
-			assertEquals(e.toString(), args[1]);
-			return true;
-		}
-	}
+        @Override
+        public boolean matches(Object argument) {
+            Object[] args = (Object[]) argument;
+            assertEquals(task, args[0]);
+            assertEquals(e.toString(), args[1]);
+            return true;
+        }
+    }
 }


[23/50] [abbrv] nifi git commit: NIFI-1082: Ensure that events returned from the provenance repository are ordered such that newest events are provided first

Posted by ma...@apache.org.
NIFI-1082: Ensure that events returned from the provenance repository are ordered such that newest events are provided first


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

Branch: refs/heads/NIFI-730
Commit: cf8ca3dc2c9ee9220c0b83b6c003ef20c67fbd33
Parents: dc4004d
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Oct 28 14:45:13 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Oct 28 17:32:51 2015 -0400

----------------------------------------------------------------------
 .../StandardProvenanceEventRecord.java          |  36 ++--
 .../nifi/provenance/StandardQueryResult.java    |  25 ++-
 .../nifi/provenance/IndexConfiguration.java     |  47 +++---
 .../PersistentProvenanceRepository.java         | 124 +++++++-------
 .../nifi/provenance/lucene/DocsReader.java      |  49 ++----
 .../nifi/provenance/lucene/IndexSearch.java     |  35 ++--
 .../nifi/provenance/lucene/LineageQuery.java    |   7 +-
 .../TestPersistentProvenanceRepository.java     | 163 ++++++++++++++++++-
 .../VolatileProvenanceRepository.java           |  16 +-
 9 files changed, 327 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
index 4eb7001..892a8f8 100644
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
+++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
@@ -71,7 +71,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
     private final Map<String, String> previousAttributes;
     private final Map<String, String> updatedAttributes;
 
-    private volatile long eventId;
+    private volatile long eventId = -1L;
 
     private StandardProvenanceEventRecord(final Builder builder) {
         this.eventTime = builder.eventTime;
@@ -105,8 +105,8 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         contentClaimOffset = builder.contentClaimOffset;
         contentSize = builder.contentSize;
 
-        previousAttributes = builder.previousAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.previousAttributes);
-        updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
+        previousAttributes = builder.previousAttributes == null ? Collections.<String, String> emptyMap() : Collections.unmodifiableMap(builder.previousAttributes);
+        updatedAttributes = builder.updatedAttributes == null ? Collections.<String, String> emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes);
 
         sourceQueueIdentifier = builder.sourceQueueIdentifier;
 
@@ -198,12 +198,12 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
 
     @Override
     public List<String> getParentUuids() {
-        return parentUuids == null ? Collections.<String>emptyList() : parentUuids;
+        return parentUuids == null ? Collections.<String> emptyList() : parentUuids;
     }
 
     @Override
     public List<String> getChildUuids() {
-        return childrenUuids == null ? Collections.<String>emptyList() : childrenUuids;
+        return childrenUuids == null ? Collections.<String> emptyList() : childrenUuids;
     }
 
     @Override
@@ -298,7 +298,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         }
 
         return -37423 + 3 * componentId.hashCode() + (transitUri == null ? 0 : 41 * transitUri.hashCode())
-                + (relationship == null ? 0 : 47 * relationship.hashCode()) + 44 * eventTypeCode;
+            + (relationship == null ? 0 : 47 * relationship.hashCode()) + 44 * eventTypeCode;
     }
 
     @Override
@@ -316,7 +316,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
         final StandardProvenanceEventRecord other = (StandardProvenanceEventRecord) obj;
         // If event ID's are populated and not equal, return false. If they have not yet been populated, do not
         // use them in the comparison.
-        if (eventId > 0L && other.getEventId() > 0L && eventId != other.getEventId()) {
+        if (eventId >= 0L && other.getEventId() >= 0L && eventId != other.getEventId()) {
             return false;
         }
         if (eventType != other.eventType) {
@@ -397,16 +397,16 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
     @Override
     public String toString() {
         return "ProvenanceEventRecord ["
-                + "eventId=" + eventId
-                + ", eventType=" + eventType
-                + ", eventTime=" + new Date(eventTime)
-                + ", uuid=" + uuid
-                + ", fileSize=" + contentSize
-                + ", componentId=" + componentId
-                + ", transitUri=" + transitUri
-                + ", sourceSystemFlowFileIdentifier=" + sourceSystemFlowFileIdentifier
-                + ", parentUuids=" + parentUuids
-                + ", alternateIdentifierUri=" + alternateIdentifierUri + "]";
+            + "eventId=" + eventId
+            + ", eventType=" + eventType
+            + ", eventTime=" + new Date(eventTime)
+            + ", uuid=" + uuid
+            + ", fileSize=" + contentSize
+            + ", componentId=" + componentId
+            + ", transitUri=" + transitUri
+            + ", sourceSystemFlowFileIdentifier=" + sourceSystemFlowFileIdentifier
+            + ", parentUuids=" + parentUuids
+            + ", alternateIdentifierUri=" + alternateIdentifierUri + "]";
     }
 
     public static class Builder implements ProvenanceEventBuilder {
@@ -663,7 +663,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
             setFlowFileEntryDate(flowFile.getEntryDate());
             setLineageIdentifiers(flowFile.getLineageIdentifiers());
             setLineageStartDate(flowFile.getLineageStartDate());
-            setAttributes(Collections.<String, String>emptyMap(), flowFile.getAttributes());
+            setAttributes(Collections.<String, String> emptyMap(), flowFile.getAttributes());
             uuid = flowFile.getAttribute(CoreAttributes.UUID.key());
             this.contentSize = flowFile.getSize();
             return this;

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
index 9a9a27d..bef63e1 100644
--- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
+++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
@@ -18,6 +18,7 @@ package org.apache.nifi.provenance;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -40,7 +41,7 @@ public class StandardQueryResult implements QueryResult {
 
     private final Lock writeLock = rwLock.writeLock();
     // guarded by writeLock
-    private final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>();
+    private final List<List<ProvenanceEventRecord>> matchingRecords;
     private long totalHitCount;
     private int numCompletedSteps = 0;
     private Date expirationDate;
@@ -53,6 +54,11 @@ public class StandardQueryResult implements QueryResult {
         this.query = query;
         this.numSteps = numSteps;
         this.creationNanos = System.nanoTime();
+        this.matchingRecords = new ArrayList<>(numSteps);
+
+        for (int i = 0; i < Math.max(1, numSteps); i++) {
+            matchingRecords.add(Collections.<ProvenanceEventRecord> emptyList());
+        }
 
         updateExpiration();
     }
@@ -61,13 +67,14 @@ public class StandardQueryResult implements QueryResult {
     public List<ProvenanceEventRecord> getMatchingEvents() {
         readLock.lock();
         try {
-            if (matchingRecords.size() <= query.getMaxResults()) {
-                return new ArrayList<>(matchingRecords);
-            }
-
             final List<ProvenanceEventRecord> copy = new ArrayList<>(query.getMaxResults());
-            for (int i = 0; i < query.getMaxResults(); i++) {
-                copy.add(matchingRecords.get(i));
+            for (final List<ProvenanceEventRecord> recordList : matchingRecords) {
+                if (copy.size() + recordList.size() > query.getMaxResults()) {
+                    copy.addAll(recordList.subList(0, query.getMaxResults() - copy.size()));
+                    return copy;
+                } else {
+                    copy.addAll(recordList);
+                }
             }
 
             return copy;
@@ -141,10 +148,10 @@ public class StandardQueryResult implements QueryResult {
         }
     }
 
-    public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits) {
+    public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits, final int indexId) {
         writeLock.lock();
         try {
-            this.matchingRecords.addAll(matchingRecords);
+            this.matchingRecords.set(indexId, new ArrayList<>(matchingRecords));
             this.totalHitCount += totalHits;
 
             numCompletedSteps++;

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java
index 4e80811..a6e6d5d 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java
@@ -36,7 +36,6 @@ import java.util.regex.Pattern;
 
 import org.apache.nifi.provenance.serialization.RecordReader;
 import org.apache.nifi.provenance.serialization.RecordReaders;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -173,6 +172,9 @@ public class IndexConfiguration {
             for (final List<File> list : indexDirectoryMap.values()) {
                 files.addAll(list);
             }
+
+            Collections.sort(files, new IndexDirectoryComparator());
+
             return files;
         } finally {
             lock.unlock();
@@ -198,11 +200,11 @@ public class IndexConfiguration {
      * span (times inclusive).
      *
      * @param startTime the start time of the query for which the indices are
-     * desired
+     *            desired
      * @param endTime the end time of the query for which the indices are
-     * desired
+     *            desired
      * @return the index directories that are applicable only for the given time
-     * span (times inclusive).
+     *         span (times inclusive).
      */
     public List<File> getIndexDirectories(final Long startTime, final Long endTime) {
         if (startTime == null && endTime == null) {
@@ -213,14 +215,7 @@ public class IndexConfiguration {
         lock.lock();
         try {
             final List<File> sortedIndexDirectories = getIndexDirectories();
-            Collections.sort(sortedIndexDirectories, new Comparator<File>() {
-                @Override
-                public int compare(final File o1, final File o2) {
-                    final long epochTimestamp1 = getIndexStartTime(o1);
-                    final long epochTimestamp2 = getIndexStartTime(o2);
-                    return Long.compare(epochTimestamp1, epochTimestamp2);
-                }
-            });
+            Collections.sort(sortedIndexDirectories, new IndexDirectoryComparator());
 
             for (final File indexDir : sortedIndexDirectories) {
                 // If the index was last modified before the start time, we know that it doesn't
@@ -252,9 +247,9 @@ public class IndexConfiguration {
      * event log
      *
      * @param provenanceLogFile the provenance log file for which the index
-     * directories are desired
+     *            directories are desired
      * @return the index directories that are applicable only for the given
-     * event log
+     *         event log
      */
     public List<File> getIndexDirectories(final File provenanceLogFile) {
         final List<File> dirs = new ArrayList<>();
@@ -262,23 +257,16 @@ public class IndexConfiguration {
         try {
             final List<File> indices = indexDirectoryMap.get(provenanceLogFile.getParentFile());
             if (indices == null) {
-                return Collections.<File>emptyList();
+                return Collections.<File> emptyList();
             }
 
             final List<File> sortedIndexDirectories = new ArrayList<>(indices);
-            Collections.sort(sortedIndexDirectories, new Comparator<File>() {
-                @Override
-                public int compare(final File o1, final File o2) {
-                    final long epochTimestamp1 = getIndexStartTime(o1);
-                    final long epochTimestamp2 = getIndexStartTime(o2);
-                    return Long.compare(epochTimestamp1, epochTimestamp2);
-                }
-            });
+            Collections.sort(sortedIndexDirectories, new IndexDirectoryComparator());
 
             final Long firstEntryTime = getFirstEntryTime(provenanceLogFile);
             if (firstEntryTime == null) {
                 logger.debug("Found no records in {} so returning no Indices for it", provenanceLogFile);
-                return Collections.<File>emptyList();
+                return Collections.<File> emptyList();
             }
 
             boolean foundIndexCreatedLater = false;
@@ -376,7 +364,7 @@ public class IndexConfiguration {
         lock.lock();
         try {
             if (minIndexedId == null || id > minIndexedId) {
-                if (maxIndexedId == null || id > maxIndexedId) {  // id will be > maxIndexedId if all records were expired
+                if (maxIndexedId == null || id > maxIndexedId) { // id will be > maxIndexedId if all records were expired
                     minIndexedId = maxIndexedId;
                 } else {
                     minIndexedId = id;
@@ -395,4 +383,13 @@ public class IndexConfiguration {
             lock.unlock();
         }
     }
+
+    private class IndexDirectoryComparator implements Comparator<File> {
+        @Override
+        public int compare(final File o1, final File o2) {
+            final long epochTimestamp1 = getIndexStartTime(o1);
+            final long epochTimestamp2 = getIndexStartTime(o2);
+            return -Long.compare(epochTimestamp1, epochTimestamp2);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
index 89e1419..7f1bf8c 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
@@ -108,7 +108,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     public static final String EVENT_CATEGORY = "Provenance Repository";
     private static final String FILE_EXTENSION = ".prov";
     private static final String TEMP_FILE_SUFFIX = ".prov.part";
-    private static final long PURGE_EVENT_MILLISECONDS = 2500L; //Determines the frequency over which the task to delete old events will occur
+    private static final long PURGE_EVENT_MILLISECONDS = 2500L; // Determines the frequency over which the task to delete old events will occur
     public static final int SERIALIZATION_VERSION = 8;
     public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
     public static final Pattern INDEX_PATTERN = Pattern.compile("index-\\d+");
@@ -404,9 +404,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 // if this is the first record, try to find out the block index and jump directly to
                 // the block index. This avoids having to read through a lot of data that we don't care about
                 // just to get to the first record that we want.
-                if ( records.isEmpty() ) {
+                if (records.isEmpty()) {
                     final TocReader tocReader = reader.getTocReader();
-                    if ( tocReader != null ) {
+                    if (tocReader != null) {
                         final Integer blockIndex = tocReader.getBlockIndexForEventId(firstRecordId);
                         if (blockIndex != null) {
                             reader.skipToBlock(blockIndex);
@@ -641,7 +641,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
 
             indexManager.close();
 
-            if ( writers != null ) {
+            if (writers != null) {
                 for (final RecordWriter writer : writers) {
                     writer.close();
                 }
@@ -700,7 +700,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                     // journal will result in corruption!
                     writer.markDirty();
                     dirtyWriterCount.incrementAndGet();
-                    streamStartTime.set(0L);    // force rollover to happen soon.
+                    streamStartTime.set(0L); // force rollover to happen soon.
                     throw t;
                 } finally {
                     writer.unlock();
@@ -912,15 +912,15 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 removed.add(baseName);
             } catch (final FileNotFoundException fnf) {
                 logger.warn("Failed to perform Expiration Action {} on Provenance Event file {} because the file no longer exists; will not "
-                        + "perform additional Expiration Actions on this file", currentAction, file);
+                    + "perform additional Expiration Actions on this file", currentAction, file);
                 removed.add(baseName);
             } catch (final Throwable t) {
                 logger.warn("Failed to perform Expiration Action {} on Provenance Event file {} due to {}; will not perform additional "
-                        + "Expiration Actions on this file at this time", currentAction, file, t.toString());
+                    + "Expiration Actions on this file at this time", currentAction, file, t.toString());
                 logger.warn("", t);
                 eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Failed to perform Expiration Action " + currentAction +
-                        " on Provenance Event file " + file + " due to " + t.toString() + "; will not perform additional Expiration Actions " +
-                        "on this file at this time");
+                    " on Provenance Event file " + file + " due to " + t.toString() + "; will not perform additional Expiration Actions " +
+                    "on this file at this time");
             }
         }
 
@@ -1131,10 +1131,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     protected int getJournalCount() {
         // determine how many 'journals' we have in the journals directories
         int journalFileCount = 0;
-        for ( final File storageDir : configuration.getStorageDirectories() ) {
+        for (final File storageDir : configuration.getStorageDirectories()) {
             final File journalsDir = new File(storageDir, "journals");
             final File[] journalFiles = journalsDir.listFiles();
-            if ( journalFiles != null ) {
+            if (journalFiles != null) {
                 journalFileCount += journalFiles.length;
             }
         }
@@ -1169,12 +1169,12 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                     writer.close();
                 } catch (final IOException ioe) {
                     logger.warn("Failed to close {} due to {}", writer, ioe.toString());
-                    if ( logger.isDebugEnabled() ) {
+                    if (logger.isDebugEnabled()) {
                         logger.warn("", ioe);
                     }
                 }
             }
-            if ( logger.isDebugEnabled() ) {
+            if (logger.isDebugEnabled()) {
                 logger.debug("Going to merge {} files for journals starting with ID {}", journalsToMerge.size(), LuceneUtil.substringBefore(journalsToMerge.get(0).getName(), "."));
             }
 
@@ -1263,10 +1263,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             // that is no longer the case.
             if (journalFileCount > journalCountThreshold || repoSize > sizeThreshold) {
                 logger.warn("The rate of the dataflow is exceeding the provenance recording rate. "
-                        + "Slowing down flow to accommodate. Currently, there are {} journal files ({} bytes) and "
-                        + "threshold for blocking is {} ({} bytes)", journalFileCount, repoSize, journalCountThreshold, sizeThreshold);
+                    + "Slowing down flow to accommodate. Currently, there are {} journal files ({} bytes) and "
+                    + "threshold for blocking is {} ({} bytes)", journalFileCount, repoSize, journalCountThreshold, sizeThreshold);
                 eventReporter.reportEvent(Severity.WARNING, "Provenance Repository", "The rate of the dataflow is "
-                        + "exceeding the provenance recording rate. Slowing down flow to accommodate");
+                    + "exceeding the provenance recording rate. Slowing down flow to accommodate");
 
                 while (journalFileCount > journalCountThreshold || repoSize > sizeThreshold) {
                     // if a shutdown happens while we are in this loop, kill the rollover thread and break
@@ -1293,15 +1293,15 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                     }
 
                     logger.debug("Provenance Repository is still behind. Keeping flow slowed down "
-                            + "to accommodate. Currently, there are {} journal files ({} bytes) and "
-                            + "threshold for blocking is {} ({} bytes)", journalFileCount, repoSize, journalCountThreshold, sizeThreshold);
+                        + "to accommodate. Currently, there are {} journal files ({} bytes) and "
+                        + "threshold for blocking is {} ({} bytes)", journalFileCount, repoSize, journalCountThreshold, sizeThreshold);
 
                     journalFileCount = getJournalCount();
                     repoSize = getSize(getLogFiles(), 0L);
                 }
 
                 logger.info("Provenance Repository has now caught up with rolling over journal files. Current number of "
-                        + "journal files to be rolled over is {}", journalFileCount);
+                    + "journal files to be rolled over is {}", journalFileCount);
             }
 
             // we've finished rolling over successfully. Create new writers and reset state.
@@ -1335,7 +1335,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             }
 
             for (final File journalFile : journalFiles) {
-                if ( journalFile.isDirectory() ) {
+                if (journalFile.isDirectory()) {
                     continue;
                 }
 
@@ -1403,7 +1403,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
      */
     File mergeJournals(final List<File> journalFiles, final File suggestedMergeFile, final EventReporter eventReporter) throws IOException {
         logger.debug("Merging {} to {}", journalFiles, suggestedMergeFile);
-        if ( this.closed.get() ) {
+        if (this.closed.get()) {
             logger.info("Provenance Repository has been closed; will not merge journal files to {}", suggestedMergeFile);
             return null;
         }
@@ -1439,14 +1439,14 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 // we have all "partial" files and there is already a merged file. Delete the data from the index
                 // because the merge file may not be fully merged. We will re-merge.
                 logger.warn("Merged Journal File {} already exists; however, all partial journal files also exist "
-                        + "so assuming that the merge did not finish. Repeating procedure in order to ensure consistency.");
+                    + "so assuming that the merge did not finish. Repeating procedure in order to ensure consistency.");
 
                 final DeleteIndexAction deleteAction = new DeleteIndexAction(this, indexConfig, indexManager);
                 try {
                     deleteAction.execute(suggestedMergeFile);
                 } catch (final Exception e) {
                     logger.warn("Failed to delete records from Journal File {} from the index; this could potentially result in duplicates. Failure was due to {}", suggestedMergeFile, e.toString());
-                    if ( logger.isDebugEnabled() ) {
+                    if (logger.isDebugEnabled()) {
                         logger.warn("", e);
                     }
                 }
@@ -1460,18 +1460,18 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 }
 
                 final File tocFile = TocUtil.getTocFile(suggestedMergeFile);
-                if ( tocFile.exists() && !tocFile.delete() ) {
+                if (tocFile.exists() && !tocFile.delete()) {
                     logger.error("Failed to delete .toc file {}; this may result in not being able to read the Provenance Events from the {} Journal File. "
                         + "This can be corrected by manually deleting the {} file", tocFile, suggestedMergeFile, tocFile);
                 }
             }
         } else {
             logger.warn("Cannot merge journal files {} because expected first file to end with extension '.0' "
-                    + "but it did not; assuming that the files were already merged but only some finished deletion "
-                    + "before restart. Deleting remaining partial journal files.", journalFiles);
+                + "but it did not; assuming that the files were already merged but only some finished deletion "
+                + "before restart. Deleting remaining partial journal files.", journalFiles);
 
-            for ( final File file : journalFiles ) {
-                if ( !file.delete() && file.exists() ) {
+            for (final File file : journalFiles) {
+                if (!file.delete() && file.exists()) {
                     logger.warn("Failed to delete unneeded journal file {}; this file should be cleaned up manually", file);
                 }
             }
@@ -1529,7 +1529,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 } catch (final EOFException eof) {
                 } catch (final Exception e) {
                     logger.warn("Failed to generate Provenance Event Record from Journal due to " + e + "; it's possible that the record wasn't "
-                            + "completely written to the file. This record will be skipped.");
+                        + "completely written to the file. This record will be skipped.");
                     if (logger.isDebugEnabled()) {
                         logger.warn("", e);
                     }
@@ -1544,11 +1544,11 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                     continue;
                 }
 
-                if ( record.getEventTime() < earliestTimestamp ) {
+                if (record.getEventTime() < earliestTimestamp) {
                     earliestTimestamp = record.getEventTime();
                 }
 
-                if ( record.getEventId() < minEventId ) {
+                if (record.getEventId() < minEventId) {
                     minEventId = record.getEventId();
                 }
 
@@ -1799,7 +1799,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         final int numQueries = querySubmissionMap.size();
         if (numQueries > MAX_UNDELETED_QUERY_RESULTS) {
             throw new IllegalStateException("Cannot process query because there are currently " + numQueries + " queries whose results have not "
-                    + "been deleted due to poorly behaving clients not issuing DELETE requests. Please try again later.");
+                + "been deleted due to poorly behaving clients not issuing DELETE requests. Please try again later.");
         }
 
         if (query.getEndDate() != null && query.getStartDate() != null && query.getStartDate().getTime() > query.getEndDate().getTime()) {
@@ -1820,7 +1820,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
 
                 Long maxEventId = getMaxEventId();
                 if (maxEventId == null) {
-                    result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+                    result.getResult().update(Collections.<ProvenanceEventRecord> emptyList(), 0L, 0);
                     maxEventId = 0L;
                 }
                 Long minIndexedId = indexConfig.getMinIdIndexed();
@@ -1830,7 +1830,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
 
                 final long totalNumDocs = maxEventId - minIndexedId;
 
-                result.getResult().update(trimmed, totalNumDocs);
+                result.getResult().update(trimmed, totalNumDocs, 0);
             } else {
                 queryExecService.submit(new GetMostRecentRunnable(query, result));
             }
@@ -1839,18 +1839,18 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             return result;
         }
 
-        final AtomicInteger retrievalCount = new AtomicInteger(0);
         final List<File> indexDirectories = indexConfig.getIndexDirectories(
-                query.getStartDate() == null ? null : query.getStartDate().getTime(),
-                        query.getEndDate() == null ? null : query.getEndDate().getTime());
+            query.getStartDate() == null ? null : query.getStartDate().getTime(),
+            query.getEndDate() == null ? null : query.getEndDate().getTime());
         final AsyncQuerySubmission result = new AsyncQuerySubmission(query, indexDirectories.size());
         querySubmissionMap.put(query.getIdentifier(), result);
 
         if (indexDirectories.isEmpty()) {
-            result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+            result.getResult().update(Collections.<ProvenanceEventRecord> emptyList(), 0L, 0);
         } else {
+            int indexId = 0;
             for (final File indexDir : indexDirectories) {
-                queryExecService.submit(new QueryRunnable(query, result, indexDir, retrievalCount));
+                queryExecService.submit(new QueryRunnable(query, result, indexDir, indexId++));
             }
         }
 
@@ -2024,11 +2024,11 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     }
 
     Lineage computeLineage(final String flowFileUuid) throws IOException {
-        return computeLineage(Collections.<String>singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
+        return computeLineage(Collections.<String> singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
     }
 
     private Lineage computeLineage(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId, final Long startTimestamp,
-            final Long endTimestamp) throws IOException {
+        final Long endTimestamp) throws IOException {
         final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, computationType, eventId, startTimestamp, endTimestamp);
         final StandardLineageResult result = submission.getResult();
         while (!result.isFinished()) {
@@ -2051,7 +2051,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     }
 
     private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final LineageComputationType computationType,
-            final Long eventId, final long startTimestamp, final long endTimestamp) {
+        final Long eventId, final long startTimestamp, final long endTimestamp) {
         final List<File> indexDirs = indexConfig.getIndexDirectories(startTimestamp, endTimestamp);
         final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexDirs.size());
         lineageSubmissionMap.put(result.getLineageIdentifier(), result);
@@ -2068,9 +2068,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         try {
             final ProvenanceEventRecord event = getEvent(eventId);
             if (event == null) {
-                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+                submission.getResult().update(Collections.<ProvenanceEventRecord> emptyList());
                 return submission;
             }
 
@@ -2081,13 +2081,13 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 case REPLAY:
                     return submitLineageComputation(event.getChildUuids(), LineageComputationType.EXPAND_CHILDREN, eventId, event.getEventTime(), Long.MAX_VALUE);
                 default:
-                    final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+                    final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
                     lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
                     submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded");
                     return submission;
             }
         } catch (final IOException ioe) {
-            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
 
             if (ioe.getMessage() == null) {
@@ -2105,9 +2105,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         try {
             final ProvenanceEventRecord event = getEvent(eventId);
             if (event == null) {
-                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+                submission.getResult().update(Collections.<ProvenanceEventRecord> emptyList());
                 return submission;
             }
 
@@ -2118,14 +2118,14 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 case REPLAY:
                     return submitLineageComputation(event.getParentUuids(), LineageComputationType.EXPAND_PARENTS, eventId, event.getLineageStartDate(), event.getEventTime());
                 default: {
-                    final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
+                    final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1);
                     lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
                     submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded");
                     return submission;
                 }
             }
         } catch (final IOException ioe) {
-            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
+            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
 
             if (ioe.getMessage() == null) {
@@ -2248,7 +2248,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             // get the max indexed event id
             final Long maxEventId = indexConfig.getMaxIdIndexed();
             if (maxEventId == null) {
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+                submission.getResult().update(Collections.<ProvenanceEventRecord> emptyList(), 0L, 0);
                 return;
             }
 
@@ -2263,7 +2263,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 final long totalNumDocs = maxEventId - minIndexedId;
 
                 final List<ProvenanceEventRecord> mostRecent = getEvents(startIndex, maxResults);
-                submission.getResult().update(mostRecent, totalNumDocs);
+                // reverse the order so that the newest events come first.
+                Collections.reverse(mostRecent);
+                submission.getResult().update(mostRecent, totalNumDocs, 0);
             } catch (final IOException ioe) {
                 logger.error("Failed to retrieve records from Provenance Repository: " + ioe.toString());
                 if (logger.isDebugEnabled()) {
@@ -2284,24 +2286,28 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         private final Query query;
         private final AsyncQuerySubmission submission;
         private final File indexDir;
-        private final AtomicInteger retrievalCount;
+        private final int indexId;
 
-        public QueryRunnable(final Query query, final AsyncQuerySubmission submission, final File indexDir, final AtomicInteger retrievalCount) {
+        public QueryRunnable(final Query query, final AsyncQuerySubmission submission, final File indexDir, final int indexId) {
             this.query = query;
             this.submission = submission;
             this.indexDir = indexDir;
-            this.retrievalCount = retrievalCount;
+            this.indexId = indexId;
         }
 
         @Override
         public void run() {
             try {
                 final IndexSearch search = new IndexSearch(PersistentProvenanceRepository.this, indexDir, indexManager, maxAttributeChars);
-                final StandardQueryResult queryResult = search.search(query, retrievalCount, firstEventTimestamp);
-                submission.getResult().update(queryResult.getMatchingEvents(), queryResult.getTotalHitCount());
+                final StandardQueryResult queryResult = search.search(query, firstEventTimestamp);
+
+                logger.debug("Merging query results for indexId {}; before merge, num events = {}", indexId, queryResult.getTotalHitCount());
+                submission.getResult().update(queryResult.getMatchingEvents(), queryResult.getTotalHitCount(), indexId);
+                logger.debug("Merging query results for indexId {}; after merge, num events = {}", indexId, queryResult.getTotalHitCount());
+
                 if (queryResult.isFinished()) {
                     logger.info("Successfully executed Query[{}] against Index {}; Search took {} milliseconds; Total Hits = {}",
-                            query, indexDir, queryResult.getQueryTime(), queryResult.getTotalHitCount());
+                        query, indexDir, queryResult.getQueryTime(), queryResult.getTotalHitCount());
                 }
             } catch (final Throwable t) {
                 logger.error("Failed to query Provenance Repository Index {} due to {}", indexDir, t.toString());
@@ -2344,7 +2350,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 result.update(matchingRecords);
 
                 logger.info("Successfully created Lineage for FlowFiles with UUIDs {} in {} milliseconds; Lineage contains {} nodes and {} edges",
-                        flowFileUuids, result.getComputationTime(TimeUnit.MILLISECONDS), result.getNodes().size(), result.getEdges().size());
+                    flowFileUuids, result.getComputationTime(TimeUnit.MILLISECONDS), result.getNodes().size(), result.getEdges().size());
             } catch (final Throwable t) {
                 logger.error("Failed to query provenance repository due to {}", t.toString());
                 if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
index c2a7609..e6e78c1 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
@@ -22,25 +22,23 @@ import java.io.IOException;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.SearchableFields;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 import org.apache.nifi.provenance.serialization.RecordReader;
 import org.apache.nifi.provenance.serialization.RecordReaders;
 import org.apache.nifi.provenance.toc.TocReader;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.TopDocs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,11 +49,7 @@ public class DocsReader {
     }
 
     public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
-            final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
-        if (retrievalCount.get() >= maxResults) {
-            return Collections.emptySet();
-        }
-
+        final int maxResults, final int maxAttributeChars) throws IOException {
         final long start = System.nanoTime();
         final int numDocs = Math.min(topDocs.scoreDocs.length, maxResults);
         final List<Document> docs = new ArrayList<>(numDocs);
@@ -68,13 +62,13 @@ public class DocsReader {
 
         final long readDocuments = System.nanoTime() - start;
         logger.debug("Reading {} Lucene Documents took {} millis", docs.size(), TimeUnit.NANOSECONDS.toMillis(readDocuments));
-        return read(docs, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars);
+        return read(docs, allProvenanceLogFiles, maxResults, maxAttributeChars);
     }
 
 
     private long getByteOffset(final Document d, final RecordReader reader) {
         final IndexableField blockField = d.getField(FieldNames.BLOCK_INDEX);
-        if ( blockField != null ) {
+        if (blockField != null) {
             final int blockIndex = blockField.numericValue().intValue();
             final TocReader tocReader = reader.getTocReader();
             return tocReader.getBlockOffset(blockIndex);
@@ -86,21 +80,21 @@ public class DocsReader {
 
     private ProvenanceEventRecord getRecord(final Document d, final RecordReader reader) throws IOException {
         final IndexableField blockField = d.getField(FieldNames.BLOCK_INDEX);
-        if ( blockField == null ) {
+        if (blockField == null) {
             reader.skipTo(getByteOffset(d, reader));
         } else {
             reader.skipToBlock(blockField.numericValue().intValue());
         }
 
         StandardProvenanceEventRecord record;
-        while ( (record = reader.nextRecord()) != null) {
+        while ((record = reader.nextRecord()) != null) {
             final IndexableField idField = d.getField(SearchableFields.Identifier.getSearchableFieldName());
-            if ( idField == null || idField.numericValue().longValue() == record.getEventId() ) {
+            if (idField == null || idField.numericValue().longValue() == record.getEventId()) {
                 break;
             }
         }
 
-        if ( record == null ) {
+        if (record == null) {
             throw new IOException("Failed to find Provenance Event " + d);
         } else {
             return record;
@@ -109,10 +103,7 @@ public class DocsReader {
 
 
     public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles,
-        final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
-        if (retrievalCount.get() >= maxResults) {
-            return Collections.emptySet();
-        }
+        final int maxResults, final int maxAttributeChars) throws IOException {
 
         LuceneUtil.sortDocsForRetrieval(docs);
 
@@ -129,7 +120,7 @@ public class DocsReader {
         try {
             for (final Document d : docs) {
                 final String storageFilename = d.getField(FieldNames.STORAGE_FILENAME).stringValue();
-                if ( storageFilesToSkip.contains(storageFilename) ) {
+                if (storageFilesToSkip.contains(storageFilename)) {
                     continue;
                 }
 
@@ -137,10 +128,6 @@ public class DocsReader {
                     if (reader != null && storageFilename.equals(lastStorageFilename)) {
                         matchingRecords.add(getRecord(d, reader));
                         eventsReadThisFile++;
-
-                        if ( retrievalCount.incrementAndGet() >= maxResults ) {
-                            break;
-                        }
                     } else {
                         logger.debug("Opening log file {}", storageFilename);
 
@@ -152,14 +139,14 @@ public class DocsReader {
                         final List<File> potentialFiles = LuceneUtil.getProvenanceLogFiles(storageFilename, allProvenanceLogFiles);
                         if (potentialFiles.isEmpty()) {
                             logger.warn("Could not find Provenance Log File with basename {} in the "
-                                    + "Provenance Repository; assuming file has expired and continuing without it", storageFilename);
+                                + "Provenance Repository; assuming file has expired and continuing without it", storageFilename);
                             storageFilesToSkip.add(storageFilename);
                             continue;
                         }
 
                         if (potentialFiles.size() > 1) {
                             throw new FileNotFoundException("Found multiple Provenance Log Files with basename " +
-                                    storageFilename + " in the Provenance Repository");
+                                storageFilename + " in the Provenance Repository");
                         }
 
                         for (final File file : potentialFiles) {
@@ -171,10 +158,6 @@ public class DocsReader {
                                 reader = RecordReaders.newRecordReader(file, allProvenanceLogFiles, maxAttributeChars);
                                 matchingRecords.add(getRecord(d, reader));
                                 eventsReadThisFile = 1;
-
-                                if ( retrievalCount.incrementAndGet() >= maxResults ) {
-                                    break;
-                                }
                             } catch (final IOException e) {
                                 throw new IOException("Failed to retrieve record " + d + " from Provenance File " + file + " due to " + e, e);
                             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
index 7fcd8ab..c0ca8a7 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
@@ -19,17 +19,22 @@ package org.apache.nifi.provenance.lucene;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.TopDocs;
 import org.apache.nifi.provenance.PersistentProvenanceRepository;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.SearchableFields;
 import org.apache.nifi.provenance.StandardQueryResult;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,7 +53,7 @@ public class IndexSearch {
         this.maxAttributeChars = maxAttributeChars;
     }
 
-    public StandardQueryResult search(final org.apache.nifi.provenance.search.Query provenanceQuery, final AtomicInteger retrievedCount, final long firstEventTimestamp) throws IOException {
+    public StandardQueryResult search(final org.apache.nifi.provenance.search.Query provenanceQuery, final long firstEventTimestamp) throws IOException {
         if (!indexDirectory.exists() && !indexDirectory.mkdirs()) {
             throw new IOException("Unable to create Indexing Directory " + indexDirectory);
         }
@@ -57,7 +62,6 @@ public class IndexSearch {
         }
 
         final StandardQueryResult sqr = new StandardQueryResult(provenanceQuery, 1);
-        final Set<ProvenanceEventRecord> matchingRecords;
 
         // we need to set the start date because if we do not, the first index may still have events that have aged off from
         // the repository, and we don't want those events to count toward the total number of matches.
@@ -77,38 +81,47 @@ public class IndexSearch {
             final long searchStartNanos = System.nanoTime();
             final long openSearcherNanos = searchStartNanos - start;
 
-            final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
+            final Sort sort = new Sort(new SortField(SearchableFields.Identifier.getSearchableFieldName(), Type.LONG, true));
+            final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults(), sort);
             final long finishSearch = System.nanoTime();
             final long searchNanos = finishSearch - searchStartNanos;
 
             logger.debug("Searching {} took {} millis; opening searcher took {} millis", this,
-                    TimeUnit.NANOSECONDS.toMillis(searchNanos), TimeUnit.NANOSECONDS.toMillis(openSearcherNanos));
+                TimeUnit.NANOSECONDS.toMillis(searchNanos), TimeUnit.NANOSECONDS.toMillis(openSearcherNanos));
 
             if (topDocs.totalHits == 0) {
-                sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+                sqr.update(Collections.<ProvenanceEventRecord> emptyList(), 0, 0);
                 return sqr;
             }
 
             final DocsReader docsReader = new DocsReader(repository.getConfiguration().getStorageDirectories());
-            matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount,
+            final Set<ProvenanceEventRecord> matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(),
                 provenanceQuery.getMaxResults(), maxAttributeChars);
 
             final long readRecordsNanos = System.nanoTime() - finishSearch;
             logger.debug("Reading {} records took {} millis for {}", matchingRecords.size(), TimeUnit.NANOSECONDS.toMillis(readRecordsNanos), this);
 
-            sqr.update(matchingRecords, topDocs.totalHits);
+            // The records returned are going to be in a sorted set. The sort order will be dependent on
+            // the ID of the events, which is also approximately the same as the timestamp of the event (i.e.
+            // it's ordered by the time when the event was inserted into the repo, not the time when the event took
+            // place). We want to reverse this so that we get the newest events first, so we have to first create a
+            // new List object to hold the events, and then reverse the list.
+            final List<ProvenanceEventRecord> recordList = new ArrayList<>(matchingRecords);
+            Collections.reverse(recordList);
+
+            sqr.update(recordList, topDocs.totalHits, 0);
             return sqr;
         } catch (final FileNotFoundException e) {
             // nothing has been indexed yet, or the data has already aged off
             logger.warn("Attempted to search Provenance Index {} but could not find the file due to {}", indexDirectory, e);
-            if ( logger.isDebugEnabled() ) {
+            if (logger.isDebugEnabled()) {
                 logger.warn("", e);
             }
 
-            sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+            sqr.update(Collections.<ProvenanceEventRecord> emptyList(), 0, 0);
             return sqr;
         } finally {
-            if ( searcher != null ) {
+            if (searcher != null) {
                 indexManager.returnIndexSearcher(indexDirectory, searcher);
             }
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
index e9e6e63..e1996f6 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
@@ -25,7 +25,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
@@ -95,11 +94,11 @@ public class LineageQuery {
 
                 final DocsReader docsReader = new DocsReader(repo.getConfiguration().getStorageDirectories());
                 final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, searcher.getIndexReader(), repo.getAllLogFiles(),
-                    new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars);
+                    Integer.MAX_VALUE, maxAttributeChars);
 
                 final long readDocsEnd = System.nanoTime();
                 logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis",
-                        indexDirectory, TimeUnit.NANOSECONDS.toMillis(searchEnd - searchStart), TimeUnit.NANOSECONDS.toMillis(readDocsEnd - searchEnd));
+                    indexDirectory, TimeUnit.NANOSECONDS.toMillis(searchEnd - searchStart), TimeUnit.NANOSECONDS.toMillis(readDocsEnd - searchEnd));
 
                 return recs;
             } finally {
@@ -108,7 +107,7 @@ public class LineageQuery {
         } catch (final FileNotFoundException fnfe) {
             // nothing has been indexed yet, or the data has already aged off
             logger.warn("Attempted to search Provenance Index {} but could not find the file due to {}", indexDirectory, fnfe);
-            if ( logger.isDebugEnabled() ) {
+            if (logger.isDebugEnabled()) {
                 logger.warn("", fnfe);
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
index 6875743..036e97f 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.FileFilter;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -117,16 +118,16 @@ public class TestPersistentProvenanceRepository {
         // Delete all of the storage files. We do this in order to clean up the tons of files that
         // we create but also to ensure that we have closed all of the file handles. If we leave any
         // streams open, for instance, this will throw an IOException, causing our unit test to fail.
-        for ( final File storageDir : config.getStorageDirectories() ) {
+        for (final File storageDir : config.getStorageDirectories()) {
             int i;
-            for (i=0; i < 3; i++) {
+            for (i = 0; i < 3; i++) {
                 try {
                     FileUtils.deleteFile(storageDir, true);
                     break;
                 } catch (final IOException ioe) {
                     // if there is a virus scanner, etc. running in the background we may not be able to
                     // delete the file. Wait a sec and try again.
-                    if ( i == 2 ) {
+                    if (i == 2) {
                         throw ioe;
                     } else {
                         try {
@@ -441,7 +442,7 @@ public class TestPersistentProvenanceRepository {
         repo.waitForRollover();
 
         final Query query = new Query(UUID.randomUUID().toString());
-        //        query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000*"));
+        // query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000*"));
         query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.Filename, "file-*"));
         query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "12?4"));
         query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
@@ -603,7 +604,7 @@ public class TestPersistentProvenanceRepository {
 
         repo.purgeOldEvents();
 
-        Thread.sleep(2000L);    // purge is async. Give it time to do its job.
+        Thread.sleep(2000L); // purge is async. Give it time to do its job.
 
         query.setMaxResults(100);
         final QuerySubmission noResultSubmission = repo.submitQuery(query);
@@ -614,6 +615,152 @@ public class TestPersistentProvenanceRepository {
         assertEquals(0, noResultSubmission.getResult().getTotalHitCount());
     }
 
+
+    @Test
+    public void testEventsAreOrdered() throws IOException, InterruptedException, ParseException {
+        final RepositoryConfiguration config = createConfiguration();
+        config.setMaxRecordLife(30, TimeUnit.SECONDS);
+        config.setMaxStorageCapacity(1024L * 1024L);
+        config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
+        config.setMaxEventFileCapacity(1024L * 1024L);
+        config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
+
+        repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
+        repo.initialize(getEventReporter());
+
+        final String uuid = "00000000-0000-0000-0000-000000000000";
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("abc", "xyz");
+        attributes.put("xyz", "abc");
+        attributes.put("filename", "file-" + uuid);
+
+        final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
+        builder.setEventTime(System.currentTimeMillis());
+        builder.setEventType(ProvenanceEventType.RECEIVE);
+        builder.setTransitUri("nifi://unit-test");
+        attributes.put("uuid", uuid);
+        builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
+        builder.setComponentId("1234");
+        builder.setComponentType("dummy processor");
+
+        for (int i = 0; i < 10; i++) {
+            builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
+            builder.setEventTime(System.currentTimeMillis());
+            repo.registerEvent(builder.build());
+            Thread.sleep(20);
+        }
+
+        // Give time for rollover to happen
+        repo.waitForRollover();
+
+        // Perform a "Most Recent Events" Query
+        final Query query = new Query(UUID.randomUUID().toString());
+        query.setMaxResults(100);
+
+        final QueryResult result = repo.queryEvents(query);
+        assertEquals(10, result.getMatchingEvents().size());
+
+        final List<ProvenanceEventRecord> matchingEvents = result.getMatchingEvents();
+        long timestamp = matchingEvents.get(0).getEventTime();
+
+        for (final ProvenanceEventRecord record : matchingEvents) {
+            assertTrue(record.getEventTime() <= timestamp);
+            timestamp = record.getEventTime();
+        }
+
+        // Perform a Query for a particular component, so that this doesn't just get the most recent events
+        // and has to actually hit Lucene.
+        final Query query2 = new Query(UUID.randomUUID().toString());
+        query2.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "1234"));
+        query2.setMaxResults(100);
+        final QueryResult result2 = repo.queryEvents(query2);
+        assertEquals(10, result2.getMatchingEvents().size());
+
+        final List<ProvenanceEventRecord> matchingEvents2 = result2.getMatchingEvents();
+        timestamp = matchingEvents2.get(0).getEventTime();
+
+        for (final ProvenanceEventRecord record : matchingEvents2) {
+            assertTrue(record.getEventTime() <= timestamp);
+            timestamp = record.getEventTime();
+        }
+    }
+
+
+    @Test
+    public void testEventsAreOrderedAcrossMultipleIndexes() throws IOException, InterruptedException, ParseException {
+        final RepositoryConfiguration config = createConfiguration();
+        config.setMaxRecordLife(30, TimeUnit.SECONDS);
+        config.setMaxStorageCapacity(1024L * 1024L);
+        config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
+        config.setMaxEventFileCapacity(1024L * 1024L);
+        config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
+        config.setDesiredIndexSize(1L);
+
+        repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
+        repo.initialize(getEventReporter());
+
+        final String uuid = "00000000-0000-0000-0000-000000000000";
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("abc", "xyz");
+        attributes.put("xyz", "abc");
+        attributes.put("filename", "file-" + uuid);
+
+        final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
+        builder.setEventTime(System.currentTimeMillis());
+        builder.setEventType(ProvenanceEventType.RECEIVE);
+        builder.setTransitUri("nifi://unit-test");
+        attributes.put("uuid", uuid);
+        builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
+        builder.setComponentId("1234");
+        builder.setComponentType("dummy processor");
+
+        for (int i = 0; i < 10; i++) {
+            builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
+            builder.setEventTime(System.currentTimeMillis());
+            repo.registerEvent(builder.build());
+            Thread.sleep(20);
+        }
+
+        // Give time for rollover to happen
+        repo.waitForRollover();
+
+        for (int i = 0; i < 10; i++) {
+            builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
+            builder.setEventTime(System.currentTimeMillis());
+            repo.registerEvent(builder.build());
+            Thread.sleep(20);
+        }
+
+        repo.waitForRollover();
+
+        // Verify that multiple indexes exist
+        final File storageDir = config.getStorageDirectories().get(0);
+        final File[] subDirs = storageDir.listFiles(new FilenameFilter() {
+            @Override
+            public boolean accept(final File dir, final String name) {
+                return name.startsWith("index-");
+            }
+        });
+        assertEquals(2, subDirs.length);
+
+        // Perform a Query for a particular component, so that this doesn't just get the most recent events
+        // and has to actually hit Lucene.
+        final Query query = new Query(UUID.randomUUID().toString());
+        query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "1234"));
+        query.setMaxResults(100);
+        final QueryResult result = repo.queryEvents(query);
+        assertEquals(20, result.getMatchingEvents().size());
+
+        final List<ProvenanceEventRecord> matchingEvents = result.getMatchingEvents();
+        long timestamp = matchingEvents.get(0).getEventTime();
+
+        for (final ProvenanceEventRecord record : matchingEvents) {
+            assertTrue(record.getEventTime() <= timestamp);
+            timestamp = record.getEventTime();
+        }
+    }
+
+
     @Test
     public void testIndexAndCompressOnRolloverAndSubsequentEmptySearch() throws IOException, InterruptedException, ParseException {
         final RepositoryConfiguration config = createConfiguration();
@@ -939,7 +1086,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
-        config.setDesiredIndexSize(10);  // force new index to be created for each rollover
+        config.setDesiredIndexSize(10); // force new index to be created for each rollover
 
         repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
         repo.initialize(getEventReporter());
@@ -961,7 +1108,7 @@ public class TestPersistentProvenanceRepository {
         for (int i = 0; i < 10; i++) {
             attributes.put("uuid", "00000000-0000-0000-0000-00000000000" + i);
             builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
-            builder.setEventTime(10L);  // make sure the events are destroyed when we call purge
+            builder.setEventTime(10L); // make sure the events are destroyed when we call purge
             repo.registerEvent(builder.build());
         }
 
@@ -1019,7 +1166,7 @@ public class TestPersistentProvenanceRepository {
     @Test
     public void testBackPressure() throws IOException, InterruptedException {
         final RepositoryConfiguration config = createConfiguration();
-        config.setMaxEventFileCapacity(1L);  // force rollover on each record.
+        config.setMaxEventFileCapacity(1L); // force rollover on each record.
         config.setJournalCount(1);
 
         final AtomicInteger journalCountRef = new AtomicInteger(0);

http://git-wip-us.apache.org/repos/asf/nifi/blob/cf8ca3dc/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
index f4f9d12..ac13f08 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
@@ -373,7 +373,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     }
 
     public Lineage computeLineage(final String flowFileUUID) throws IOException {
-        return computeLineage(Collections.<String>singleton(flowFileUUID), LineageComputationType.FLOWFILE_LINEAGE, null);
+        return computeLineage(Collections.<String> singleton(flowFileUUID), LineageComputationType.FLOWFILE_LINEAGE, null);
     }
 
     private Lineage computeLineage(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId) throws IOException {
@@ -411,9 +411,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     public ComputeLineageSubmission submitExpandParents(final long eventId) {
         final ProvenanceEventRecord event = getEvent(eventId);
         if (event == null) {
-            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
+            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-            submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+            submission.getResult().update(Collections.<ProvenanceEventRecord> emptyList());
             return submission;
         }
 
@@ -424,7 +424,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
             case CLONE:
                 return submitLineageComputation(event.getParentUuids(), LineageComputationType.EXPAND_PARENTS, eventId);
             default: {
-                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
+                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
                 submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded");
                 return submission;
@@ -440,9 +440,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     public ComputeLineageSubmission submitExpandChildren(final long eventId) {
         final ProvenanceEventRecord event = getEvent(eventId);
         if (event == null) {
-            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+            final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-            submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+            submission.getResult().update(Collections.<ProvenanceEventRecord> emptyList());
             return submission;
         }
 
@@ -453,7 +453,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
             case CLONE:
                 return submitLineageComputation(event.getChildUuids(), LineageComputationType.EXPAND_CHILDREN, eventId);
             default: {
-                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
+                final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
                 submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded");
                 return submission;
@@ -526,7 +526,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
 
             }, IterationDirection.BACKWARD);
 
-            submission.getResult().update(matchingRecords, matchingCount.get());
+            submission.getResult().update(matchingRecords, matchingCount.get(), 0);
         }
     }
 


[16/50] [abbrv] nifi git commit: NIFI-10: Updated FetchHDFS, FetchFileTransfer to use new FETCH provenance event

Posted by ma...@apache.org.
NIFI-10: Updated FetchHDFS, FetchFileTransfer to use new FETCH provenance event


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

Branch: refs/heads/NIFI-730
Commit: 1c1738670ca5f921863572434b6f4cc4daddad38
Parents: 1700633
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Oct 26 17:14:29 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 17:14:29 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java    | 2 +-
 .../org/apache/nifi/processors/standard/FetchFileTransfer.java    | 3 +--
 2 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/1c173867/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index aa03e73..c27ade9 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -111,7 +111,7 @@ public class FetchHDFS extends AbstractHadoopProcessor {
             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.getProvenanceReporter().fetch(flowFile, uri.toString(), 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});

http://git-wip-us.apache.org/repos/asf/nifi/blob/1c173867/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
index a405afb..f3fa347 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -278,8 +278,7 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         flowFile = session.putAllAttributes(flowFile, attributes);
 
         // emit provenance event and transfer FlowFile
-        session.getProvenanceReporter().modifyContent(flowFile, "Content replaced with content from " + protocolName + "://" + host + ":" + port + "/" + filename,
-            stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+        session.getProvenanceReporter().fetch(flowFile, protocolName + "://" + host + ":" + port + "/" + filename, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
         session.transfer(flowFile, REL_SUCCESS);
 
         // it is critical that we commit the session before moving/deleting the remote file. Otherwise, we could have a situation where


[09/50] [abbrv] nifi git commit: NIFI-1056: Do not treat BigDecimal, BigInteger the same way we treat Number's when serializing to Avro because Avro Java Library doesn't support them

Posted by ma...@apache.org.
NIFI-1056: Do not treat BigDecimal, BigInteger the same way we treat Number's when serializing to Avro because Avro Java Library doesn't support them

Reviewed and Unit Test Amended by Tony Kurc (tkurc@apache.org)


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

Branch: refs/heads/NIFI-730
Commit: 8d2f9bc64b03702f4db53de6ac741ce102413bfa
Parents: 385bfbb
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 25 11:31:30 2015 -0400
Committer: Tony Kurc <tr...@gmail.com>
Committed: Sun Oct 25 11:31:30 2015 -0400

----------------------------------------------------------------------
 .../processors/standard/util/JdbcCommon.java    |  6 +++
 .../standard/util/TestJdbcCommon.java           | 45 ++++++++++++++++++++
 2 files changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/8d2f9bc6/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
index 59eaf30..ac95c8b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java
@@ -46,6 +46,8 @@ import static java.sql.Types.VARCHAR;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -99,6 +101,10 @@ public class JdbcCommon {
                         // org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
                         rec.put(i - 1, ((Byte) value).intValue());
 
+                    } else if (value instanceof BigDecimal || value instanceof BigInteger) {
+                        // Avro can't handle BigDecimal and BigInteger as numbers - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
+                        rec.put(i - 1, value.toString());
+
                     } else if (value instanceof Number || value instanceof Boolean) {
                         rec.put(i - 1, value);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/8d2f9bc6/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
index 9c9532f..b8fcfed 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestJdbcCommon.java
@@ -25,6 +25,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.Field;
+import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -34,6 +35,7 @@ import java.sql.Statement;
 import java.sql.Types;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileStream;
@@ -44,6 +46,8 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestJdbcCommon {
 
@@ -180,6 +184,47 @@ public class TestJdbcCommon {
     }
 
 
+    @Test
+    public void testConvertToAvroStreamForBigDecimal() throws SQLException, IOException {
+        final ResultSetMetaData metadata = Mockito.mock(ResultSetMetaData.class);
+        Mockito.when(metadata.getColumnCount()).thenReturn(1);
+        Mockito.when(metadata.getColumnType(1)).thenReturn(Types.NUMERIC);
+        Mockito.when(metadata.getColumnName(1)).thenReturn("Chairman");
+        Mockito.when(metadata.getTableName(1)).thenReturn("table");
+
+        final ResultSet rs = Mockito.mock(ResultSet.class);
+        Mockito.when(rs.getMetaData()).thenReturn(metadata);
+
+        final AtomicInteger counter = new AtomicInteger(1);
+        Mockito.doAnswer(new Answer<Boolean>() {
+            @Override
+            public Boolean answer(InvocationOnMock invocation) throws Throwable {
+                return counter.getAndDecrement() > 0;
+            }
+        }).when(rs).next();
+
+        final BigDecimal bigDecimal = new BigDecimal(38D);
+        Mockito.when(rs.getObject(Mockito.anyInt())).thenReturn(bigDecimal);
+
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        JdbcCommon.convertToAvroStream(rs, baos);
+
+        final byte[] serializedBytes = baos.toByteArray();
+
+        final InputStream instream = new ByteArrayInputStream(serializedBytes);
+
+        final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
+        try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
+            GenericRecord record = null;
+            while (dataFileReader.hasNext()) {
+                record = dataFileReader.next(record);
+                assertEquals(bigDecimal.toString(), record.get("Chairman").toString());
+            }
+        }
+    }
+
+
     // many test use Derby as database, so ensure driver is available
     @Test
     public void testDriverLoad() throws ClassNotFoundException {


[35/50] [abbrv] nifi git commit: NIFI-869 Fixed formatting issue Fixed formatting issue with printed error message which only apears when NiFi is cnfigured using Logback. Please see NIFI-869 for more details

Posted by ma...@apache.org.
NIFI-869 Fixed formatting issue Fixed formatting issue with printed error message which only apears when NiFi is cnfigured using Logback. Please see NIFI-869 for more details

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: 90aea0135062c50e2c8635e178188d2d8c35b883
Parents: dc4004d
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Oct 29 14:19:44 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Oct 30 15:52:33 2015 -0400

----------------------------------------------------------------------
 .../nifi/controller/scheduling/StandardProcessScheduler.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/90aea013/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 9ff58c8..5d99d07 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -655,7 +655,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                             final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
 
                             final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service);
-                            componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
+                            componentLog.error("failed to invoke @OnEnabled method due to {}", new Object[]{cause.toString()});
                             LOG.error("Failed to invoke @OnEnabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString());
                             if (LOG.isDebugEnabled()) {
                                 LOG.error("", cause);


[49/50] [abbrv] nifi git commit: Merge branch 'master' into NIFI-730

Posted by ma...@apache.org.
Merge branch 'master' into NIFI-730


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

Branch: refs/heads/NIFI-730
Commit: e4cebba3c7868010c2b9fd994b850f40b81aa044
Parents: 5a04021 a549621
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Nov 2 14:02:16 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 14:02:16 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/action/Action.java     |  50 ++
 .../java/org/apache/nifi/action/Component.java  |  34 +
 .../java/org/apache/nifi/action/Operation.java  |  37 +
 .../component/details/ComponentDetails.java     |  26 +
 .../component/details/ExtensionDetails.java     |  26 +
 .../details/RemoteProcessGroupDetails.java      |  26 +
 .../nifi/action/details/ActionDetails.java      |  26 +
 .../nifi/action/details/ConfigureDetails.java   |  30 +
 .../nifi/action/details/ConnectDetails.java     |  40 +
 .../apache/nifi/action/details/MoveDetails.java |  30 +
 .../nifi/action/details/PurgeDetails.java       |  28 +
 .../annotation/behavior/InputRequirement.java   |  67 ++
 .../apache/nifi/processor/ProcessSession.java   |  27 +
 .../nifi/provenance/ProvenanceEventType.java    |  31 +-
 .../nifi/provenance/ProvenanceReporter.java     |  37 +
 .../org/apache/nifi/reporting/EventAccess.java  |  13 +
 .../StandardProvenanceEventRecord.java          |  36 +-
 .../nifi/provenance/StandardQueryResult.java    |  25 +-
 .../nifi/processor/util/StandardValidators.java |  10 +
 .../client/socket/EndpointConnectionPool.java   |   3 +-
 .../src/main/asciidoc/administration-guide.adoc |   4 +-
 .../src/main/asciidoc/developer-guide.adoc      |  11 +
 .../src/main/asciidoc/getting-started.adoc      | 754 +++++++++++++++++
 .../src/main/asciidoc/images/add-processor.png  | Bin 31524 -> 92164 bytes
 nifi-docs/src/main/asciidoc/user-guide.adoc     |   2 +-
 .../src/main/java/MyProcessor.java              |  11 +-
 .../org/apache/nifi/util/MockEventAccess.java   |  27 +
 .../apache/nifi/util/MockProcessSession.java    |   8 +
 .../nifi/util/MockProvenanceReporter.java       |  35 +-
 .../nifi/processors/avro/ConvertAvroToJSON.java |  70 +-
 .../processors/avro/ExtractAvroMetadata.java    |  29 +-
 .../apache/nifi/processors/avro/SplitAvro.java  |  27 +-
 .../processors/avro/TestConvertAvroToJSON.java  |  47 +-
 .../processors/aws/AbstractAWSProcessor.java    |   2 +-
 .../nifi/processors/aws/s3/DeleteS3Object.java  |  98 +++
 .../nifi/processors/aws/s3/FetchS3Object.java   |   5 +-
 .../nifi/processors/aws/s3/PutS3Object.java     |  48 +-
 .../apache/nifi/processors/aws/sns/PutSNS.java  |   3 +
 .../nifi/processors/aws/sqs/DeleteSQS.java      |   3 +
 .../apache/nifi/processors/aws/sqs/GetSQS.java  |   5 +-
 .../apache/nifi/processors/aws/sqs/PutSQS.java  |   5 +-
 .../org.apache.nifi.processor.Processor         |   1 +
 .../processors/aws/s3/TestDeleteS3Object.java   | 141 ++++
 .../nifi/processors/flume/ExecuteFlumeSink.java |  14 +-
 .../processors/flume/ExecuteFlumeSource.java    |  14 +-
 .../org/apache/nifi/admin/dao/ActionDAO.java    |  12 +-
 .../java/org/apache/nifi/admin/dao/UserDAO.java |   3 +-
 .../nifi/admin/dao/impl/StandardActionDAO.java  |  93 ++-
 .../nifi/admin/dao/impl/StandardUserDAO.java    |   4 +-
 .../apache/nifi/admin/service/AuditService.java |  18 +-
 .../admin/service/action/AddActionsAction.java  |   3 +-
 .../service/action/PurgeActionsAction.java      |   3 +-
 .../service/impl/StandardAuditService.java      |  24 +-
 .../resources/nifi-administration-context.xml   |   2 +-
 .../apache/nifi/web/api/dto/ProcessorDTO.java   |  15 +
 .../manager/impl/ClusteredEventAccess.java      |  21 +-
 .../cluster/manager/impl/WebClusterManager.java |   2 +-
 .../apache/nifi/controller/ProcessorNode.java   |   7 +-
 .../apache/nifi/controller/FlowController.java  |  15 +-
 .../nifi/controller/FlowUnmarshaller.java       |  77 --
 .../nifi/controller/StandardProcessorNode.java  | 109 ++-
 .../apache/nifi/controller/TemplateManager.java |   1 +
 .../repository/BatchingSessionFactory.java      |   5 +
 .../repository/FileSystemRepository.java        |  40 +-
 .../repository/StandardProcessSession.java      |  12 +
 .../repository/StandardProvenanceReporter.java  |  31 +-
 .../nifi/processor/SimpleProcessLogger.java     |  20 +-
 .../nifi/spring/FlowControllerFactoryBean.java  |   8 +
 .../src/main/resources/FlowConfiguration.xsd    |   2 +-
 .../src/main/resources/nifi-context.xml         |   1 +
 .../controller/StandardFlowServiceTest.java     |   5 +-
 .../repository/TestFileSystemRepository.java    |  18 +
 .../repository/TestStandardProcessSession.java  |  39 +-
 .../nifi/processor/TestSimpleProcessLogger.java | 101 +++
 .../src/main/resources/bin/nifi.sh              |  96 +--
 .../nifi-framework/nifi-user-actions/pom.xml    |   7 +
 .../java/org/apache/nifi/action/Action.java     | 121 ---
 .../java/org/apache/nifi/action/Component.java  |  34 -
 .../apache/nifi/action/FlowChangeAction.java    | 130 +++
 .../java/org/apache/nifi/action/Operation.java  |  37 -
 .../component/details/ComponentDetails.java     |  26 -
 .../component/details/ExtensionDetails.java     |  34 -
 .../details/FlowChangeExtensionDetails.java     |  35 +
 .../FlowChangeRemoteProcessGroupDetails.java    |  35 +
 .../details/RemoteProcessGroupDetails.java      |  34 -
 .../nifi/action/details/ActionDetails.java      |  26 -
 .../nifi/action/details/ConfigureDetails.java   |  52 --
 .../nifi/action/details/ConnectDetails.java     |  90 --
 .../details/FlowChangeConfigureDetails.java     |  55 ++
 .../details/FlowChangeConnectDetails.java       |  97 +++
 .../action/details/FlowChangeMoveDetails.java   |  65 ++
 .../action/details/FlowChangePurgeDetails.java  |  46 +
 .../apache/nifi/action/details/MoveDetails.java |  61 --
 .../nifi/action/details/PurgeDetails.java       |  45 -
 .../apache/nifi/audit/ControllerAuditor.java    |  27 +-
 .../nifi/audit/ControllerServiceAuditor.java    |  43 +-
 .../org/apache/nifi/audit/FunnelAuditor.java    |   7 +-
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  25 +-
 .../java/org/apache/nifi/audit/PortAuditor.java |  27 +-
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  25 +-
 .../org/apache/nifi/audit/ProcessorAuditor.java |  25 +-
 .../apache/nifi/audit/RelationshipAuditor.java  |  18 +-
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  35 +-
 .../apache/nifi/audit/ReportingTaskAuditor.java |  25 +-
 .../org/apache/nifi/audit/SnippetAuditor.java   |  26 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |   9 +-
 .../StandardNiFiWebConfigurationContext.java    |  13 +-
 .../apache/nifi/web/StandardNiFiWebContext.java |  13 +-
 .../nifi/web/api/ApplicationResource.java       |   5 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  21 +-
 .../partials/provenance/provenance-content.jsp  |   2 +
 .../nifi-web-ui/src/main/webapp/css/main.css    |   7 +-
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |  20 +-
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js |  13 +-
 .../js/nf/canvas/nf-connection-configuration.js |   8 +-
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  23 +-
 .../js/nf/provenance/nf-provenance-table.js     |  54 +-
 .../webapp/js/nf/provenance/nf-provenance.js    |  16 +-
 .../org/apache/nifi/processors/GeoEnrichIP.java |   3 +
 .../hadoop/AbstractHadoopProcessor.java         |  92 +-
 .../hadoop/CreateHadoopSequenceFile.java        |   4 +-
 .../nifi/processors/hadoop/FetchHDFS.java       |   5 +-
 .../apache/nifi/processors/hadoop/GetHDFS.java  |   3 +
 .../apache/nifi/processors/hadoop/ListHDFS.java |   3 +
 .../apache/nifi/processors/hadoop/PutHDFS.java  |   3 +
 .../processors/hl7/ExtractHL7Attributes.java    |   3 +
 .../apache/nifi/processors/hl7/RouteHL7.java    |   3 +
 .../processors/image/ExtractImageMetadata.java  |  36 +-
 .../nifi/processors/image/ResizeImage.java      |  38 +-
 .../apache/nifi/processors/kafka/GetKafka.java  |  21 +-
 .../apache/nifi/processors/kafka/PutKafka.java  |  16 +-
 .../processors/kite/AbstractKiteProcessor.java  |  11 +-
 .../nifi/processors/kite/ConvertCSVToAvro.java  | 312 +++----
 .../nifi/processors/kite/ConvertJSONToAvro.java |  14 +-
 .../processors/kite/StoreInKiteDataset.java     |   9 +-
 .../processors/kite/TestCSVToAvroProcessor.java |  39 +
 .../nifi/processors/yandex/YandexTranslate.java |   3 +
 .../nifi/provenance/IndexConfiguration.java     |  47 +-
 .../PersistentProvenanceRepository.java         | 124 +--
 .../nifi/provenance/lucene/DocsReader.java      |  49 +-
 .../nifi/provenance/lucene/IndexSearch.java     |  35 +-
 .../nifi/provenance/lucene/LineageQuery.java    |   7 +-
 .../TestPersistentProvenanceRepository.java     | 163 +++-
 .../VolatileProvenanceRepository.java           |  16 +-
 .../nifi/processors/twitter/GetTwitter.java     |   5 +-
 .../apache/nifi/processors/solr/GetSolr.java    |  43 +-
 .../processors/solr/PutSolrContentStream.java   |  33 +-
 .../nifi-standard-prioritizers/pom.xml          |   4 +
 .../PriorityAttributePrioritizer.java           |   7 +-
 .../PriorityAttributePrioritizerTest.java       |  17 +-
 .../nifi-standard-processors/pom.xml            |   8 +
 .../standard/AbstractListProcessor.java         | 507 +++++++++++
 .../processors/standard/AttributesToJSON.java   | 242 ++++++
 .../standard/Base64EncodeContent.java           |  31 +-
 .../processors/standard/CompressContent.java    |  15 +-
 .../nifi/processors/standard/ControlRate.java   | 113 +--
 .../standard/ConvertCharacterSet.java           |  22 +-
 .../processors/standard/ConvertJSONToSQL.java   |   3 +
 .../processors/standard/DetectDuplicate.java    |   3 +
 .../processors/standard/DistributeLoad.java     |   3 +
 .../processors/standard/DuplicateFlowFile.java  |   3 +
 .../nifi/processors/standard/EncodeContent.java |  15 +-
 .../processors/standard/EncryptContent.java     |   3 +
 .../processors/standard/EvaluateJsonPath.java   |  38 +-
 .../nifi/processors/standard/EvaluateXPath.java |  29 +-
 .../processors/standard/EvaluateXQuery.java     |  25 +-
 .../processors/standard/ExecuteProcess.java     |   3 +
 .../nifi/processors/standard/ExecuteSQL.java    |  12 +-
 .../standard/ExecuteStreamCommand.java          |   7 +-
 .../nifi/processors/standard/ExtractText.java   |   3 +
 .../processors/standard/FetchFileTransfer.java  | 346 ++++++++
 .../nifi/processors/standard/FetchSFTP.java     |  93 +++
 .../processors/standard/GenerateFlowFile.java   |  11 +-
 .../apache/nifi/processors/standard/GetFTP.java |  13 +-
 .../nifi/processors/standard/GetFile.java       |   7 +-
 .../nifi/processors/standard/GetHTTP.java       |   3 +
 .../nifi/processors/standard/GetJMSQueue.java   |   3 +
 .../nifi/processors/standard/GetJMSTopic.java   |   3 +
 .../nifi/processors/standard/GetSFTP.java       |   7 +-
 .../processors/standard/HandleHttpRequest.java  |  30 +-
 .../processors/standard/HandleHttpResponse.java |   5 +-
 .../nifi/processors/standard/HashAttribute.java |   5 +-
 .../nifi/processors/standard/HashContent.java   |   5 +-
 .../processors/standard/IdentifyMimeType.java   |   5 +-
 .../nifi/processors/standard/InvokeHTTP.java    | 201 ++---
 .../nifi/processors/standard/JmsConsumer.java   |   2 +
 .../processors/standard/ListFileTransfer.java   | 119 +++
 .../nifi/processors/standard/ListSFTP.java      |  88 ++
 .../nifi/processors/standard/ListenHTTP.java    | 121 +--
 .../nifi/processors/standard/ListenUDP.java     |  18 +-
 .../nifi/processors/standard/LogAttribute.java  |  16 +-
 .../nifi/processors/standard/MergeContent.java  |  17 +-
 .../nifi/processors/standard/ModifyBytes.java   |  14 +-
 .../processors/standard/MonitorActivity.java    |  31 +-
 .../nifi/processors/standard/PostHTTP.java      |   3 +
 .../standard/PutDistributedMapCache.java        | 252 ++++++
 .../nifi/processors/standard/PutEmail.java      |   3 +
 .../apache/nifi/processors/standard/PutFTP.java |   3 +
 .../nifi/processors/standard/PutFile.java       |   3 +
 .../apache/nifi/processors/standard/PutJMS.java |   7 +-
 .../nifi/processors/standard/PutSFTP.java       |   3 +
 .../apache/nifi/processors/standard/PutSQL.java | 159 ++--
 .../nifi/processors/standard/ReplaceText.java   | 485 ++++++++---
 .../standard/ReplaceTextWithMapping.java        |  18 +-
 .../processors/standard/RouteOnAttribute.java   |   3 +
 .../processors/standard/RouteOnContent.java     |  19 +-
 .../nifi/processors/standard/ScanAttribute.java |  19 +-
 .../nifi/processors/standard/ScanContent.java   |   5 +-
 .../processors/standard/SegmentContent.java     |   7 +-
 .../nifi/processors/standard/SplitContent.java  |   7 +-
 .../nifi/processors/standard/SplitJson.java     |  32 +-
 .../nifi/processors/standard/SplitText.java     |  53 +-
 .../nifi/processors/standard/SplitXml.java      |  18 +-
 .../nifi/processors/standard/TransformXml.java  |   3 +
 .../nifi/processors/standard/UnpackContent.java |   9 +-
 .../nifi/processors/standard/ValidateXml.java   |  16 +-
 .../servlets/ContentAcknowledgmentServlet.java  |   3 +-
 .../standard/servlets/ListenHTTPServlet.java    |   8 +-
 .../processors/standard/util/EntityListing.java |  71 ++
 .../processors/standard/util/FTPTransfer.java   | 145 ++--
 .../nifi/processors/standard/util/FileInfo.java |  18 +-
 .../processors/standard/util/FileTransfer.java  | 337 ++++----
 .../processors/standard/util/JdbcCommon.java    |  76 +-
 .../processors/standard/util/JmsFactory.java    |  79 +-
 .../processors/standard/util/JmsProperties.java |   8 +
 .../standard/util/ListableEntity.java           |  40 +
 .../util/PermissionDeniedException.java         |  32 +
 .../processors/standard/util/SFTPTransfer.java  | 191 +++--
 .../org.apache.nifi.processor.Processor         |   4 +
 .../standard/TestAbstractListProcessor.java     | 221 +++++
 .../standard/TestAttributesToJSON.java          | 282 +++++++
 .../standard/TestConvertCharacterSet.java       |  64 +-
 .../standard/TestFetchFileTransfer.java         | 317 +++++++
 .../nifi/processors/standard/TestGetFile.java   |  21 +-
 .../processors/standard/TestInvokeHTTP.java     | 607 +-------------
 .../processors/standard/TestInvokeHttpSSL.java  |  90 ++
 .../standard/TestPutDistributedMapCache.java    | 277 +++++++
 .../nifi/processors/standard/TestPutSQL.java    |  46 +-
 .../processors/standard/TestReplaceText.java    | 635 +++++++++++++-
 .../standard/TestReplaceTextLineByLine.java     | 336 --------
 .../standard/util/TestInvokeHttpCommon.java     | 830 +++++++++++++++++++
 .../standard/util/TestJdbcCommon.java           |  87 ++
 .../standard/util/TestJdbcTypesDerby.java       | 133 +++
 .../standard/util/TestJdbcTypesH2.java          | 149 ++++
 .../AppendLineByLineTest.txt                    |  11 +
 .../PrependLineByLineTest.txt                   |  11 +
 .../processors/attributes/UpdateAttribute.java  |   9 +-
 pom.xml                                         |   2 +-
 248 files changed, 10506 insertions(+), 3447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --cc nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 20f2642,135025e..701459c
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@@ -16,40 -16,11 +16,39 @@@
   */
  package org.apache.nifi.controller;
  
 -import com.sun.jersey.api.client.ClientHandlerException;
 -import org.apache.commons.lang3.StringUtils;
 +import static java.util.Objects.requireNonNull;
 +
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.text.DateFormat;
 +import java.text.SimpleDateFormat;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Date;
 +import java.util.HashSet;
 +import java.util.LinkedHashSet;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.UUID;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ConcurrentMap;
 +import java.util.concurrent.ScheduledExecutorService;
 +import java.util.concurrent.ScheduledFuture;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicInteger;
 +import java.util.concurrent.atomic.AtomicReference;
 +import java.util.concurrent.locks.Lock;
 +import java.util.concurrent.locks.ReentrantReadWriteLock;
 +
 +import javax.net.ssl.SSLContext;
 +
- import org.apache.commons.lang3.StringUtils;
- import org.apache.nifi.admin.service.UserService;
+ import org.apache.nifi.action.Action;
 -import org.apache.nifi.admin.service.AuditService;
 -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;

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/nifi/blob/e4cebba3/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
----------------------------------------------------------------------


[47/50] [abbrv] nifi git commit: Merge branch 'NIFI-1082'

Posted by ma...@apache.org.
Merge branch 'NIFI-1082'


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

Branch: refs/heads/NIFI-730
Commit: 814e8b212cf6cb239418ea25b6f5dd1eb919ae14
Parents: abf2c62 cf8ca3d
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Nov 2 11:29:52 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 11:29:52 2015 -0500

----------------------------------------------------------------------
 .../StandardProvenanceEventRecord.java          |  36 ++--
 .../nifi/provenance/StandardQueryResult.java    |  25 ++-
 .../nifi/provenance/IndexConfiguration.java     |  47 +++---
 .../PersistentProvenanceRepository.java         | 124 +++++++-------
 .../nifi/provenance/lucene/DocsReader.java      |  49 ++----
 .../nifi/provenance/lucene/IndexSearch.java     |  35 ++--
 .../nifi/provenance/lucene/LineageQuery.java    |   7 +-
 .../TestPersistentProvenanceRepository.java     | 163 ++++++++++++++++++-
 .../VolatileProvenanceRepository.java           |  16 +-
 9 files changed, 327 insertions(+), 175 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] nifi git commit: Merge branch 'NIFI-977'

Posted by ma...@apache.org.
Merge branch 'NIFI-977'


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

Branch: refs/heads/NIFI-730
Commit: dc4004de64d10ba82298eb1f6ecb5924fd0afb62
Parents: 4c0cf7d 84db372
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Oct 27 16:53:38 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Oct 27 16:53:38 2015 -0400

----------------------------------------------------------------------
 .../apache/nifi/processors/standard/PutSQL.java | 156 +++++++++----------
 .../nifi/processors/standard/TestPutSQL.java    |  46 +++++-
 2 files changed, 119 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/dc4004de/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
----------------------------------------------------------------------
diff --cc nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
index 0913f86,5c2bbc2..c2056fe
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSQL.java
@@@ -70,27 -68,26 +70,27 @@@ import org.apache.nifi.stream.io.Stream
  
  @SupportsBatching
  @SeeAlso(ConvertJSONToSQL.class)
 +@InputRequirement(Requirement.INPUT_REQUIRED)
  @Tags({"sql", "put", "rdbms", "database", "update", "insert", "relational"})
  @CapabilityDescription("Executes a SQL UPDATE or INSERT command. The content of an incoming FlowFile is expected to be the SQL command "
-         + "to execute. The SQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
-         + "with the naming convention sql.args.N.type and sql.args.N.value, where N is a positive integer. The sql.args.N.type is expected to be "
-         + "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
+     + "to execute. The SQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
+     + "with the naming convention sql.args.N.type and sql.args.N.value, where N is a positive integer. The sql.args.N.type is expected to be "
+     + "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
  @ReadsAttributes({
-     @ReadsAttribute(attribute="fragment.identifier", description="If the <Support Fragment Transactions> property is true, this attribute is used to determine whether or "
-             + "not two FlowFiles belong to the same transaction."),
-     @ReadsAttribute(attribute="fragment.count", description="If the <Support Fragment Transactions> property is true, this attribute is used to determine how many FlowFiles "
-             + "are needed to complete the transaction."),
-     @ReadsAttribute(attribute="fragment.index", description="If the <Support Fragment Transactions> property is true, this attribute is used to determine the order that the FlowFiles "
-             + "in a transaction should be evaluated."),
-     @ReadsAttribute(attribute="sql.args.N.type", description="Incoming FlowFiles are expected to be parameterized SQL statements. The type of each Parameter is specified as an integer "
-             + "that represents the JDBC Type of the parameter."),
-     @ReadsAttribute(attribute="sql.args.N.value", description="Incoming FlowFiles are expected to be parameterized SQL statements. The value of the Parameters are specified as "
-             + "sql.args.1.value, sql.args.2.value, sql.args.3.value, and so on. The type of the sql.args.1.value Parameter is specified by the sql.args.1.type attribute.")
+     @ReadsAttribute(attribute = "fragment.identifier", description = "If the <Support Fragment Transactions> property is true, this attribute is used to determine whether or "
+         + "not two FlowFiles belong to the same transaction."),
+     @ReadsAttribute(attribute = "fragment.count", description = "If the <Support Fragment Transactions> property is true, this attribute is used to determine how many FlowFiles "
+         + "are needed to complete the transaction."),
+     @ReadsAttribute(attribute = "fragment.index", description = "If the <Support Fragment Transactions> property is true, this attribute is used to determine the order that the FlowFiles "
+         + "in a transaction should be evaluated."),
+     @ReadsAttribute(attribute = "sql.args.N.type", description = "Incoming FlowFiles are expected to be parameterized SQL statements. The type of each Parameter is specified as an integer "
+         + "that represents the JDBC Type of the parameter."),
+     @ReadsAttribute(attribute = "sql.args.N.value", description = "Incoming FlowFiles are expected to be parameterized SQL statements. The value of the Parameters are specified as "
+         + "sql.args.1.value, sql.args.2.value, sql.args.3.value, and so on. The type of the sql.args.1.value Parameter is specified by the sql.args.1.type attribute.")
  })
  @WritesAttributes({
-     @WritesAttribute(attribute="sql.generated.key", description="If the database generated a key for an INSERT statement and the Obtain Generated Keys property is set to true, "
-             + "this attribute will be added to indicate the generated key, if possible. This feature is not supported by all database vendors.")
+     @WritesAttribute(attribute = "sql.generated.key", description = "If the database generated a key for an INSERT statement and the Obtain Generated Keys property is set to true, "
+         + "this attribute will be added to indicate the generated key, if possible. This feature is not supported by all database vendors.")
  })
  public class PutSQL extends AbstractProcessor {
  


[30/50] [abbrv] nifi git commit: NIFI-1079 Create a new Attribute from the existing FlowFile attributes by taking either all of the existing attributes or a user defined list. The existing Attributes are converted to JSON and placed in a new Attribute on

Posted by ma...@apache.org.
NIFI-1079 Create a new Attribute from the existing FlowFile attributes by taking
either all of the existing attributes or a user defined list. The
existing Attributes are converted to JSON and placed in a new Attribute
on the existing FlowFile as Attribute “JSONAttributes”

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/NIFI-730
Commit: 19b7a4cc7de9da3366aa9c1a15b97c683242b4d0
Parents: 5cc2b04
Author: Jeremy Dyer <jd...@gmail.com>
Authored: Wed Oct 28 16:20:48 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Oct 30 14:29:29 2015 -0400

----------------------------------------------------------------------
 .../processors/standard/AttributesToJSON.java   | 117 ++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../standard/TestAttributesToJSON.java          | 177 +++++++++++++++++++
 3 files changed, 295 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/19b7a4cc/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
new file mode 100644
index 0000000..7098b6e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.standard;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.*;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.*;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.*;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@Tags({"JSON", "attributes"})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@CapabilityDescription("Evaluates the attributes from a FlowFile and generates a JSON string with the attribute key/value pair. " +
+        "The resulting JSON string is placed in the FlowFile as a new Attribute with the name 'JSONAttributes'. By default all" +
+        "Attributes in the FlowFile are placed in the resulting JSON string. If only certain Attributes are desired you may" +
+        "specify a list of FlowFile Attributes that you want in the resulting JSON string")
+@WritesAttribute(attribute = "JSONAttributes", description = "JSON string of all the pre-existing attributes in the flowfile")
+public class AttributesToJSON extends AbstractProcessor {
+
+    public static final String JSON_ATTRIBUTE_NAME = "JSONAttribute";
+    private static final String AT_LIST_SEPARATOR = ",";
+    private static final String DEFAULT_VALUE_IF_ATT_NOT_PRESENT = "";
+
+    public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder()
+            .name("Attributes List")
+            .description("Comma separated list of attributes to be included in the '" + JSON_ATTRIBUTE_NAME +"' attribute. This list of attributes is case sensitive. If a specified attribute is not found" +
+                    "in the flowfile an empty string will be output for that attritbute in the resulting JSON")
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("'" + JSON_ATTRIBUTE_NAME + "' attribute has been successfully added to the flowfile").build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+            .description("Failed to add '" + JSON_ATTRIBUTE_NAME + "' attribute to the flowfile").build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private static final ObjectMapper objectMapper = new ObjectMapper();
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(ATTRIBUTES_LIST);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final FlowFile original = session.get();
+        if (original == null) {
+            return;
+        }
+
+        final String atList = context.getProperty(ATTRIBUTES_LIST).getValue();
+        Map<String, String> atsToWrite = null;
+
+        //If list of attributes specified get only those attributes. Otherwise write them all
+        if (atList != null && !StringUtils.isEmpty(atList)) {
+            atsToWrite = new HashMap<>();
+            String[] ats = StringUtils.split(atList, AT_LIST_SEPARATOR);
+            if (ats != null) {
+                for (String str : ats) {
+                    String cleanStr = str.trim();
+                    String val = original.getAttribute(cleanStr);
+                    if (val != null) {
+                        atsToWrite.put(cleanStr, val);
+                    } else {
+                        atsToWrite.put(cleanStr, DEFAULT_VALUE_IF_ATT_NOT_PRESENT);
+                    }
+                }
+            }
+        } else {
+            atsToWrite = original.getAttributes();
+        }
+
+        if (atsToWrite != null) {
+            if (atsToWrite.size() == 0) {
+                getLogger().debug("Flowfile contains no attributes to convert to JSON");
+            } else {
+                try {
+                    FlowFile updated = session.putAttribute(original, JSON_ATTRIBUTE_NAME, objectMapper.writeValueAsString(atsToWrite));
+                    session.transfer(updated, REL_SUCCESS);
+                } catch (JsonProcessingException e) {
+                    getLogger().error(e.getMessage());
+                    session.transfer(original, REL_FAILURE);
+                }
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/19b7a4cc/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index b12fb6f..8507c96 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -12,6 +12,7 @@
 # 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.standard.AttributesToJSON
 org.apache.nifi.processors.standard.Base64EncodeContent
 org.apache.nifi.processors.standard.CompressContent
 org.apache.nifi.processors.standard.ControlRate

http://git-wip-us.apache.org/repos/asf/nifi/blob/19b7a4cc/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
new file mode 100644
index 0000000..a057d15
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
@@ -0,0 +1,177 @@
+package org.apache.nifi.processors.standard;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+
+
+public class TestAttributesToJSON {
+
+    private static Logger LOGGER;
+
+    static {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.AttributesToJSON", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestAttributesToJSON", "debug");
+        LOGGER = LoggerFactory.getLogger(TestAttributesToJSON.class);
+    }
+
+    private static final String TEST_ATTRIBUTE_KEY = "TestAttribute";
+    private static final String TEST_ATTRIBUTE_VALUE = "TestValue";
+
+    @Test(expected = AssertionError.class)
+    public void testInvalidUserSuppliedAttributeList() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+
+        //Attribute list CANNOT be empty
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, "");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+    }
+
+    @Test
+    public void testInvalidJSONValueInAttribute() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        //Create attribute that contains an invalid JSON Character
+        ff = session.putAttribute(ff, TEST_ATTRIBUTE_KEY, "'badjson'");
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+    }
+
+
+    @Test
+    public void testAttribuets_emptyListUserSpecifiedAttributes() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        ff = session.putAttribute(ff, TEST_ATTRIBUTE_KEY, TEST_ATTRIBUTE_VALUE);
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+        assertTrue(val.get(TEST_ATTRIBUTE_KEY).equals(TEST_ATTRIBUTE_VALUE));
+    }
+
+
+    @Test
+    public void testAttribute_singleUserDefinedAttribute() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, TEST_ATTRIBUTE_KEY);
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+        ff = session.putAttribute(ff, TEST_ATTRIBUTE_KEY, TEST_ATTRIBUTE_VALUE);
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+        assertTrue(val.get(TEST_ATTRIBUTE_KEY).equals(TEST_ATTRIBUTE_VALUE));
+        assertTrue(val.size() == 1);
+    }
+
+
+    @Test
+    public void testAttribute_singleUserDefinedAttributeWithWhiteSpace() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, " " + TEST_ATTRIBUTE_KEY + " ");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+        ff = session.putAttribute(ff, TEST_ATTRIBUTE_KEY, TEST_ATTRIBUTE_VALUE);
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+        assertTrue(val.get(TEST_ATTRIBUTE_KEY).equals(TEST_ATTRIBUTE_VALUE));
+        assertTrue(val.size() == 1);
+    }
+
+
+    @Test
+    public void testAttribute_singleNonExistingUserDefinedAttribute() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, "NonExistingAttribute");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+        ff = session.putAttribute(ff, TEST_ATTRIBUTE_KEY, TEST_ATTRIBUTE_VALUE);
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+
+        //If a Attribute is requested but does not exist then it is placed in the JSON with an empty string
+        assertTrue(val.get("NonExistingAttribute").equals(""));
+        assertTrue(val.size() == 1);
+    }
+
+}


[45/50] [abbrv] nifi git commit: Merge branch 'NIFI-1051' of https://github.com/olegz/nifi into NIFI-1051

Posted by ma...@apache.org.
Merge branch 'NIFI-1051' of https://github.com/olegz/nifi into NIFI-1051


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

Branch: refs/heads/NIFI-730
Commit: 7a73867c46629ff20ef5d84f1167cb34e503bc75
Parents: 2ae4902 5c4042b
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Nov 2 10:51:43 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 10:51:43 2015 -0500

----------------------------------------------------------------------
 .../repository/FileSystemRepository.java          |  6 ++++++
 .../repository/TestFileSystemRepository.java      | 18 ++++++++++++++++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/7a73867c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------


[27/50] [abbrv] nifi git commit: NIFI-986 Refactoring of action classes from nifi-user-actions to have interfaces in nifi-api, and adding getFlowChanges to EventAccess - Fixing empty java docs and adding sort by id asc to the history query - Changing use

Posted by ma...@apache.org.
NIFI-986 Refactoring of action classes from nifi-user-actions to have interfaces in nifi-api, and adding getFlowChanges to EventAccess
- Fixing empty java docs and adding sort by id asc to the history query
- Changing userDn to userIdentity in Action and FlowChangeAction
- Modifying NiFiAuditor to always save events locally, and implementing getFlowChanges for ClusteredEventAccess


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

Branch: refs/heads/NIFI-730
Commit: 5cc2b04b918a0a3c76c0f1d71c7a416d0dce3f39
Parents: c4f0cb1
Author: Bryan Bende <bb...@apache.org>
Authored: Fri Sep 25 13:35:57 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Thu Oct 29 16:28:36 2015 -0400

----------------------------------------------------------------------
 .../java/org/apache/nifi/action/Action.java     |  50 +++++++
 .../java/org/apache/nifi/action/Component.java  |  34 +++++
 .../java/org/apache/nifi/action/Operation.java  |  37 +++++
 .../component/details/ComponentDetails.java     |  26 ++++
 .../component/details/ExtensionDetails.java     |  26 ++++
 .../details/RemoteProcessGroupDetails.java      |  26 ++++
 .../nifi/action/details/ActionDetails.java      |  26 ++++
 .../nifi/action/details/ConfigureDetails.java   |  30 ++++
 .../nifi/action/details/ConnectDetails.java     |  40 ++++++
 .../apache/nifi/action/details/MoveDetails.java |  30 ++++
 .../nifi/action/details/PurgeDetails.java       |  28 ++++
 .../org/apache/nifi/reporting/EventAccess.java  |  13 ++
 .../org/apache/nifi/util/MockEventAccess.java   |  27 ++++
 .../org/apache/nifi/admin/dao/ActionDAO.java    |  12 +-
 .../java/org/apache/nifi/admin/dao/UserDAO.java |   3 +-
 .../nifi/admin/dao/impl/StandardActionDAO.java  |  93 ++++++++-----
 .../nifi/admin/dao/impl/StandardUserDAO.java    |   4 +-
 .../apache/nifi/admin/service/AuditService.java |  18 ++-
 .../admin/service/action/AddActionsAction.java  |   3 +-
 .../service/action/PurgeActionsAction.java      |   3 +-
 .../service/impl/StandardAuditService.java      |  24 +++-
 .../resources/nifi-administration-context.xml   |   2 +-
 .../manager/impl/ClusteredEventAccess.java      |  21 ++-
 .../cluster/manager/impl/WebClusterManager.java |   2 +-
 .../apache/nifi/controller/FlowController.java  | 136 ++++++++++---------
 .../nifi/spring/FlowControllerFactoryBean.java  |   8 ++
 .../src/main/resources/nifi-context.xml         |   1 +
 .../controller/StandardFlowServiceTest.java     |   5 +-
 .../nifi-framework/nifi-user-actions/pom.xml    |   7 +
 .../java/org/apache/nifi/action/Action.java     | 121 -----------------
 .../java/org/apache/nifi/action/Component.java  |  34 -----
 .../apache/nifi/action/FlowChangeAction.java    | 130 ++++++++++++++++++
 .../java/org/apache/nifi/action/Operation.java  |  37 -----
 .../component/details/ComponentDetails.java     |  26 ----
 .../component/details/ExtensionDetails.java     |  34 -----
 .../details/FlowChangeExtensionDetails.java     |  35 +++++
 .../FlowChangeRemoteProcessGroupDetails.java    |  35 +++++
 .../details/RemoteProcessGroupDetails.java      |  34 -----
 .../nifi/action/details/ActionDetails.java      |  26 ----
 .../nifi/action/details/ConfigureDetails.java   |  52 -------
 .../nifi/action/details/ConnectDetails.java     |  90 ------------
 .../details/FlowChangeConfigureDetails.java     |  55 ++++++++
 .../details/FlowChangeConnectDetails.java       |  97 +++++++++++++
 .../action/details/FlowChangeMoveDetails.java   |  65 +++++++++
 .../action/details/FlowChangePurgeDetails.java  |  46 +++++++
 .../apache/nifi/action/details/MoveDetails.java |  61 ---------
 .../nifi/action/details/PurgeDetails.java       |  45 ------
 .../apache/nifi/audit/ControllerAuditor.java    |  27 ++--
 .../nifi/audit/ControllerServiceAuditor.java    |  43 +++---
 .../org/apache/nifi/audit/FunnelAuditor.java    |   7 +-
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  25 ++--
 .../java/org/apache/nifi/audit/PortAuditor.java |  27 ++--
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  25 ++--
 .../org/apache/nifi/audit/ProcessorAuditor.java |  25 ++--
 .../apache/nifi/audit/RelationshipAuditor.java  |  18 +--
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  35 ++---
 .../apache/nifi/audit/ReportingTaskAuditor.java |  25 ++--
 .../org/apache/nifi/audit/SnippetAuditor.java   |  26 ++--
 .../nifi/web/StandardNiFiServiceFacade.java     |   9 +-
 .../StandardNiFiWebConfigurationContext.java    |  13 +-
 .../apache/nifi/web/StandardNiFiWebContext.java |  13 +-
 .../nifi/web/api/ApplicationResource.java       |   5 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  20 ++-
 63 files changed, 1256 insertions(+), 845 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/Action.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/Action.java b/nifi-api/src/main/java/org/apache/nifi/action/Action.java
new file mode 100644
index 0000000..4ab167e
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/Action.java
@@ -0,0 +1,50 @@
+/*
+ * 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.action;
+
+import org.apache.nifi.action.component.details.ComponentDetails;
+import org.apache.nifi.action.details.ActionDetails;
+
+import java.io.Serializable;
+import java.util.Date;
+
+/**
+ * An action taken on the flow by a user.
+ */
+public interface Action extends Serializable {
+
+    Integer getId();
+
+    Date getTimestamp();
+
+    String getUserIdentity();
+
+    String getUserName();
+
+    String getSourceId();
+
+    String getSourceName();
+
+    Component getSourceType();
+
+    ComponentDetails getComponentDetails();
+
+    Operation getOperation();
+
+    ActionDetails getActionDetails();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/Component.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/Component.java b/nifi-api/src/main/java/org/apache/nifi/action/Component.java
new file mode 100644
index 0000000..dcd23ee
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/Component.java
@@ -0,0 +1,34 @@
+/*
+ * 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.action;
+
+/**
+ * Defines possible components for a given action.
+ */
+public enum Component {
+
+    Controller,
+    Processor,
+    InputPort,
+    OutputPort,
+    ProcessGroup,
+    RemoteProcessGroup,
+    Funnel,
+    Connection,
+    ControllerService,
+    ReportingTask;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/Operation.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/Operation.java b/nifi-api/src/main/java/org/apache/nifi/action/Operation.java
new file mode 100644
index 0000000..d1bb43f
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/Operation.java
@@ -0,0 +1,37 @@
+/*
+ * 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.action;
+
+/**
+ * Defines possible operations for a given action.
+ */
+public enum Operation {
+
+    Add,
+    Remove,
+    Paste,
+    Configure,
+    Move,
+    Disconnect,
+    Connect,
+    Start,
+    Stop,
+    Enable,
+    Disable,
+    Batch,
+    Purge;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
new file mode 100644
index 0000000..1eb35c1
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
@@ -0,0 +1,26 @@
+/*
+ * 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.action.component.details;
+
+import java.io.Serializable;
+
+/**
+ * Base interface for providing component details to an Action.
+ */
+public interface ComponentDetails extends Serializable {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
new file mode 100644
index 0000000..5726c1b
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
@@ -0,0 +1,26 @@
+/*
+ * 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.action.component.details;
+
+/**
+ * Provides details of an extension on an Action.
+ */
+public interface ExtensionDetails extends ComponentDetails {
+
+    String getType();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
new file mode 100644
index 0000000..62f8628
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
@@ -0,0 +1,26 @@
+/*
+ * 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.action.component.details;
+
+/**
+ * Provides details of a remote process group to an Action.
+ */
+public interface RemoteProcessGroupDetails extends ComponentDetails {
+
+    String getUri();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/details/ActionDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/details/ActionDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/details/ActionDetails.java
new file mode 100644
index 0000000..75ebd9d
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/details/ActionDetails.java
@@ -0,0 +1,26 @@
+/*
+ * 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.action.details;
+
+import java.io.Serializable;
+
+/**
+ * Provides additional details about a given action.
+ */
+public interface ActionDetails extends Serializable {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
new file mode 100644
index 0000000..ff8db40
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
@@ -0,0 +1,30 @@
+/*
+ * 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.action.details;
+
+/**
+ * Provides details about a configure action.
+ */
+public interface ConfigureDetails extends ActionDetails {
+
+    String getName();
+
+    String getPreviousValue();
+
+    String getValue();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/details/ConnectDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
new file mode 100644
index 0000000..3c8c91b
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
@@ -0,0 +1,40 @@
+/*
+ * 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.action.details;
+
+import org.apache.nifi.action.Component;
+
+/**
+ * Provides details about a connect action.
+ */
+public interface ConnectDetails extends ActionDetails {
+
+    String getSourceId();
+
+    String getSourceName();
+
+    Component getSourceType();
+
+    String getDestinationId();
+
+    String getDestinationName();
+
+    Component getDestinationType();
+
+    String getRelationship();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/details/MoveDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/details/MoveDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/details/MoveDetails.java
new file mode 100644
index 0000000..0533b3b
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/details/MoveDetails.java
@@ -0,0 +1,30 @@
+/*
+ * 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.action.details;
+
+/**
+ * Provides details about a move action.
+ */
+public interface MoveDetails extends ActionDetails {
+    String getGroup();
+
+    String getGroupId();
+
+    String getPreviousGroup();
+
+    String getPreviousGroupId();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/action/details/PurgeDetails.java b/nifi-api/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
new file mode 100644
index 0000000..57bc57e
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
@@ -0,0 +1,28 @@
+/*
+ * 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.action.details;
+
+import java.util.Date;
+
+/**
+ * Provides details about a purge action.
+ */
+public interface PurgeDetails extends ActionDetails {
+
+    Date getEndDate();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-api/src/main/java/org/apache/nifi/reporting/EventAccess.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/EventAccess.java b/nifi-api/src/main/java/org/apache/nifi/reporting/EventAccess.java
index bdc23c2..70dedfa 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/EventAccess.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/EventAccess.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.reporting;
 
+import org.apache.nifi.action.Action;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
@@ -46,4 +47,16 @@ public interface EventAccess {
      * @return the Provenance Event Repository
      */
     ProvenanceEventRepository getProvenanceRepository();
+
+    /**
+     * Obtains flow changes starting with (and including) the given action ID. If no action
+     * exists with that ID, the first action to be returned will have an ID greater than
+     * <code>firstActionId</code>.
+     *
+     * @param firstActionId the id of the first action to obtain
+     * @param maxActions the maximum number of actions to obtain
+     * @return actions with ids greater than or equal to firstActionID, up to the max number of actions
+     */
+    List<Action> getFlowChanges(int firstActionId, final int maxActions);
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-mock/src/main/java/org/apache/nifi/util/MockEventAccess.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockEventAccess.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockEventAccess.java
index b5f6b11..8b3bf25 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockEventAccess.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockEventAccess.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.nifi.action.Action;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
@@ -29,6 +30,7 @@ public class MockEventAccess implements EventAccess {
 
     private ProcessGroupStatus processGroupStatus;
     private final List<ProvenanceEventRecord> provenanceRecords = new ArrayList<>();
+    private final List<Action> flowChanges = new ArrayList<>();
 
     public void setProcessGroupStatus(final ProcessGroupStatus status) {
         this.processGroupStatus = status;
@@ -67,4 +69,29 @@ public class MockEventAccess implements EventAccess {
     public ProvenanceEventRepository getProvenanceRepository() {
         return null;
     }
+
+    @Override
+    public List<Action> getFlowChanges(int firstActionId, int maxActions) {
+        if (firstActionId < 0 || maxActions < 1) {
+            throw new IllegalArgumentException();
+        }
+
+        final List<Action> actions = new ArrayList<>();
+
+        for (final Action action : flowChanges) {
+            if (action.getId() >= firstActionId) {
+                actions.add(action);
+                if (actions.size() >= maxActions) {
+                    return actions;
+                }
+            }
+        }
+
+        return actions;
+    }
+
+    public void addFlowChange(final Action action) {
+        this.flowChanges.add(action);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
index 925dc80..335a888 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java
@@ -16,14 +16,15 @@
  */
 package org.apache.nifi.admin.dao;
 
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
 import org.apache.nifi.action.Action;
-import org.apache.nifi.history.HistoryQuery;
 import org.apache.nifi.history.History;
+import org.apache.nifi.history.HistoryQuery;
 import org.apache.nifi.history.PreviousValue;
 
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Action data access.
  */
@@ -33,9 +34,10 @@ public interface ActionDAO {
      * Persists the specified action.
      *
      * @param action to persist
+     * @return the created Action with it's id
      * @throws DataAccessException if unable to persist
      */
-    void createAction(Action action) throws DataAccessException;
+    Action createAction(Action action) throws DataAccessException;
 
     /**
      * Finds all actions that meet the specified criteria.

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
index 6339e5a..7e91c07 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java
@@ -80,8 +80,9 @@ public interface UserDAO {
      * Creates a new user based off the specified NiFiUser.
      *
      * @param user to create
+     * @return the created user with it's id
      */
-    void createUser(NiFiUser user) throws DataAccessException;
+    NiFiUser createUser(NiFiUser user) throws DataAccessException;
 
     /**
      * Updates the specified NiFiUser.

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
index bb655eb..fe693e1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
@@ -30,13 +30,20 @@ import java.util.List;
 import java.util.Map;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.component.details.ComponentDetails;
 import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails;
 import org.apache.nifi.action.component.details.RemoteProcessGroupDetails;
 import org.apache.nifi.action.details.ActionDetails;
 import org.apache.nifi.action.details.ConfigureDetails;
 import org.apache.nifi.action.details.ConnectDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConnectDetails;
+import org.apache.nifi.action.details.FlowChangeMoveDetails;
+import org.apache.nifi.action.details.FlowChangePurgeDetails;
 import org.apache.nifi.action.details.MoveDetails;
 import org.apache.nifi.action.details.PurgeDetails;
 import org.apache.nifi.admin.RepositoryUtils;
@@ -195,8 +202,8 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     @Override
-    public void createAction(Action action) throws DataAccessException {
-        if (action.getUserDn() == null) {
+    public Action createAction(Action action) throws DataAccessException {
+        if (action.getUserIdentity() == null) {
             throw new IllegalArgumentException("User cannot be null.");
         }
 
@@ -209,7 +216,7 @@ public class StandardActionDAO implements ActionDAO {
         try {
             // obtain a statement to insert to the action table
             statement = connection.prepareStatement(INSERT_ACTION, Statement.RETURN_GENERATED_KEYS);
-            statement.setString(1, StringUtils.left(action.getUserDn(), 255));
+            statement.setString(1, StringUtils.left(action.getUserIdentity(), 255));
             statement.setString(2, StringUtils.left(action.getUserName(), 100));
             statement.setString(3, action.getSourceId());
             statement.setString(4, StringUtils.left(action.getSourceName(), 1000));
@@ -220,10 +227,21 @@ public class StandardActionDAO implements ActionDAO {
             // insert the action
             int updateCount = statement.executeUpdate();
 
+            final FlowChangeAction createdAction = new FlowChangeAction();
+            createdAction.setUserIdentity(action.getUserIdentity());
+            createdAction.setUserName(action.getUserName());
+            createdAction.setSourceId(action.getSourceId());
+            createdAction.setSourceName(action.getSourceName());
+            createdAction.setSourceType(action.getSourceType());
+            createdAction.setOperation(action.getOperation());
+            createdAction.setTimestamp(action.getTimestamp());
+            createdAction.setActionDetails(action.getActionDetails());
+            createdAction.setComponentDetails(action.getComponentDetails());
+
             // get the action id
             rs = statement.getGeneratedKeys();
             if (updateCount == 1 && rs.next()) {
-                action.setId(rs.getInt(1));
+                createdAction.setId(rs.getInt(1));
             } else {
                 throw new DataAccessException("Unable to insert action.");
             }
@@ -232,25 +250,26 @@ public class StandardActionDAO implements ActionDAO {
             statement.close();
 
             // determine the type of component
-            ComponentDetails componentDetails = action.getComponentDetails();
-            if (componentDetails instanceof ExtensionDetails) {
-                createExtensionDetails(action.getId(), (ExtensionDetails) componentDetails);
-            } else if (componentDetails instanceof RemoteProcessGroupDetails) {
-                createRemoteProcessGroupDetails(action.getId(), (RemoteProcessGroupDetails) componentDetails);
+            ComponentDetails componentDetails = createdAction.getComponentDetails();
+            if (componentDetails instanceof FlowChangeExtensionDetails) {
+                createExtensionDetails(createdAction.getId(), (ExtensionDetails) componentDetails);
+            } else if (componentDetails instanceof FlowChangeRemoteProcessGroupDetails) {
+                createRemoteProcessGroupDetails(createdAction.getId(), (RemoteProcessGroupDetails) componentDetails);
             }
 
             // determine the type of action
-            ActionDetails details = action.getActionDetails();
-            if (details instanceof ConnectDetails) {
-                createConnectDetails(action.getId(), (ConnectDetails) details);
-            } else if (details instanceof MoveDetails) {
-                createMoveDetails(action.getId(), (MoveDetails) details);
-            } else if (details instanceof ConfigureDetails) {
-                createConfigureDetails(action.getId(), (ConfigureDetails) details);
-            } else if (details instanceof PurgeDetails) {
-                createPurgeDetails(action.getId(), (PurgeDetails) details);
-            }
-
+            ActionDetails details = createdAction.getActionDetails();
+            if (details instanceof FlowChangeConnectDetails) {
+                createConnectDetails(createdAction.getId(), (ConnectDetails) details);
+            } else if (details instanceof FlowChangeMoveDetails) {
+                createMoveDetails(createdAction.getId(), (MoveDetails) details);
+            } else if (details instanceof FlowChangeConfigureDetails) {
+                createConfigureDetails(createdAction.getId(), (ConfigureDetails) details);
+            } else if (details instanceof FlowChangePurgeDetails) {
+                createPurgeDetails(createdAction.getId(), (PurgeDetails) details);
+            }
+
+            return createdAction;
         } catch (SQLException sqle) {
             throw new DataAccessException(sqle);
         } finally {
@@ -540,9 +559,9 @@ public class StandardActionDAO implements ActionDAO {
                 final Operation operation = Operation.valueOf(rs.getString("OPERATION"));
                 final Component component = Component.valueOf(rs.getString("SOURCE_TYPE"));
 
-                Action action = new Action();
+                FlowChangeAction action = new FlowChangeAction();
                 action.setId(actionId);
-                action.setUserDn(rs.getString("USER_DN"));
+                action.setUserIdentity(rs.getString("USER_DN"));
                 action.setUserName(rs.getString("USER_NAME"));
                 action.setOperation(Operation.valueOf(rs.getString("OPERATION")));
                 action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
@@ -597,7 +616,7 @@ public class StandardActionDAO implements ActionDAO {
 
     @Override
     public Action getAction(Integer actionId) throws DataAccessException {
-        Action action = null;
+        FlowChangeAction action = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -614,9 +633,9 @@ public class StandardActionDAO implements ActionDAO {
                 Component component = Component.valueOf(rs.getString("SOURCE_TYPE"));
 
                 // populate the action
-                action = new Action();
+                action = new FlowChangeAction();
                 action.setId(rs.getInt("ID"));
-                action.setUserDn(rs.getString("USER_DN"));
+                action.setUserIdentity(rs.getString("USER_DN"));
                 action.setUserName(rs.getString("USER_NAME"));
                 action.setOperation(operation);
                 action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime()));
@@ -664,7 +683,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private ExtensionDetails getExtensionDetails(Integer actionId) throws DataAccessException {
-        ExtensionDetails extensionDetails = null;
+        FlowChangeExtensionDetails extensionDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -677,7 +696,7 @@ public class StandardActionDAO implements ActionDAO {
 
             // ensure results
             if (rs.next()) {
-                extensionDetails = new ExtensionDetails();
+                extensionDetails = new FlowChangeExtensionDetails();
                 extensionDetails.setType(rs.getString("TYPE"));
             }
         } catch (SQLException sqle) {
@@ -691,7 +710,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private RemoteProcessGroupDetails getRemoteProcessGroupDetails(Integer actionId) throws DataAccessException {
-        RemoteProcessGroupDetails remoteProcessGroupDetails = null;
+        FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -704,7 +723,7 @@ public class StandardActionDAO implements ActionDAO {
 
             // ensure results
             if (rs.next()) {
-                remoteProcessGroupDetails = new RemoteProcessGroupDetails();
+                remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails();
                 remoteProcessGroupDetails.setUri(rs.getString("URI"));
             }
         } catch (SQLException sqle) {
@@ -718,7 +737,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private MoveDetails getMoveDetails(Integer actionId) throws DataAccessException {
-        MoveDetails moveDetails = null;
+        FlowChangeMoveDetails moveDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -731,7 +750,7 @@ public class StandardActionDAO implements ActionDAO {
 
             // ensure results
             if (rs.next()) {
-                moveDetails = new MoveDetails();
+                moveDetails = new FlowChangeMoveDetails();
                 moveDetails.setGroupId(rs.getString("GROUP_ID"));
                 moveDetails.setGroup(rs.getString("GROUP_NAME"));
                 moveDetails.setPreviousGroupId(rs.getString("PREVIOUS_GROUP_ID"));
@@ -748,7 +767,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private ConnectDetails getConnectDetails(Integer actionId) throws DataAccessException {
-        ConnectDetails connectionDetails = null;
+        FlowChangeConnectDetails connectionDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -764,7 +783,7 @@ public class StandardActionDAO implements ActionDAO {
                 final Component sourceComponent = Component.valueOf(rs.getString("SOURCE_TYPE"));
                 final Component destinationComponent = Component.valueOf(rs.getString("DESTINATION_TYPE"));
 
-                connectionDetails = new ConnectDetails();
+                connectionDetails = new FlowChangeConnectDetails();
                 connectionDetails.setSourceId(rs.getString("SOURCE_ID"));
                 connectionDetails.setSourceName(rs.getString("SOURCE_NAME"));
                 connectionDetails.setSourceType(sourceComponent);
@@ -784,7 +803,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private ConfigureDetails getConfigureDetails(Integer actionId) throws DataAccessException {
-        ConfigureDetails configurationDetails = null;
+        FlowChangeConfigureDetails configurationDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -797,7 +816,7 @@ public class StandardActionDAO implements ActionDAO {
 
             // ensure results
             if (rs.next()) {
-                configurationDetails = new ConfigureDetails();
+                configurationDetails = new FlowChangeConfigureDetails();
                 configurationDetails.setName(rs.getString("NAME"));
                 configurationDetails.setValue(rs.getString("VALUE"));
                 configurationDetails.setPreviousValue(rs.getString("PREVIOUS_VALUE"));
@@ -813,7 +832,7 @@ public class StandardActionDAO implements ActionDAO {
     }
 
     private PurgeDetails getPurgeDetails(Integer actionId) throws DataAccessException {
-        PurgeDetails purgeDetails = null;
+        FlowChangePurgeDetails purgeDetails = null;
         PreparedStatement statement = null;
         ResultSet rs = null;
         try {
@@ -826,7 +845,7 @@ public class StandardActionDAO implements ActionDAO {
 
             // ensure results
             if (rs.next()) {
-                purgeDetails = new PurgeDetails();
+                purgeDetails = new FlowChangePurgeDetails();
                 purgeDetails.setEndDate(new Date(rs.getTimestamp("END_DATE").getTime()));
             }
         } catch (SQLException sqle) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
index ea7c1a1..f6a62df 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java
@@ -462,7 +462,7 @@ public class StandardUserDAO implements UserDAO {
     }
 
     @Override
-    public void createUser(NiFiUser user) throws DataAccessException {
+    public NiFiUser createUser(NiFiUser user) throws DataAccessException {
         if (user.getDn() == null) {
             throw new IllegalArgumentException("User dn must be specified.");
         }
@@ -493,6 +493,8 @@ public class StandardUserDAO implements UserDAO {
             } else {
                 throw new DataAccessException("Unable to insert user.");
             }
+
+            return user;
         } catch (SQLException sqle) {
             throw new DataAccessException(sqle);
         } catch (DataAccessException dae) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
index 680173f..e2c7f77 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java
@@ -16,14 +16,15 @@
  */
 package org.apache.nifi.admin.service;
 
+import org.apache.nifi.action.Action;
+import org.apache.nifi.history.History;
+import org.apache.nifi.history.HistoryQuery;
+import org.apache.nifi.history.PreviousValue;
+
 import java.util.Collection;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
-import org.apache.nifi.action.Action;
-import org.apache.nifi.history.HistoryQuery;
-import org.apache.nifi.history.History;
-import org.apache.nifi.history.PreviousValue;
 
 /**
  * Allows NiFi actions to be audited.
@@ -55,6 +56,15 @@ public interface AuditService {
     History getActions(HistoryQuery actionQuery);
 
     /**
+     * Get the actions starting with firstActionId, returning up to maxActions.
+     *
+     * @param firstActionId the offset
+     * @param maxActions the number of actions to return
+     * @return history of actions matching the above conditions
+     */
+    History getActions(final int firstActionId, final int maxActions);
+
+    /**
      * Get the details for the specified action id. If the action cannot be
      * found, null is returned.
      *

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
index 5a2159f..db1d8a2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java
@@ -16,12 +16,13 @@
  */
 package org.apache.nifi.admin.service.action;
 
-import java.util.Collection;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.admin.dao.ActionDAO;
 import org.apache.nifi.admin.dao.DAOFactory;
 import org.apache.nifi.authorization.AuthorityProvider;
 
+import java.util.Collection;
+
 /**
  * Adds the specified actions.
  */

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
index b5a2883..6928e0d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java
@@ -16,12 +16,13 @@
  */
 package org.apache.nifi.admin.service.action;
 
-import java.util.Date;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.admin.dao.ActionDAO;
 import org.apache.nifi.admin.dao.DAOFactory;
 import org.apache.nifi.authorization.AuthorityProvider;
 
+import java.util.Date;
+
 /**
  * Purges actions up to a specified end date.
  */

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
index e588841..8f37d8e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java
@@ -16,12 +16,6 @@
  */
 package org.apache.nifi.admin.service.impl;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.admin.dao.DataAccessException;
 import org.apache.nifi.admin.service.AdministrationException;
@@ -40,6 +34,13 @@ import org.apache.nifi.history.PreviousValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 /**
  *
  */
@@ -141,6 +142,17 @@ public class StandardAuditService implements AuditService {
     }
 
     @Override
+    public History getActions(int firstActionId, int maxActions) {
+        final HistoryQuery query = new HistoryQuery();
+        query.setOffset(firstActionId);
+        query.setCount(maxActions);
+        query.setSortOrder("asc");
+        query.setSortColumn("timestamp");
+
+        return getActions(query);
+    }
+
+    @Override
     public Action getAction(Integer actionId) {
         Transaction transaction = null;
         Action action = null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
index a36619f..8cb4b97 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/resources/nifi-administration-context.xml
@@ -23,7 +23,7 @@
     http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd">
 
     <!-- user authority provider -->
-    <bean id="authorityProvider" class="org.apache.nifi.authorization.AuthorityProviderFactoryBean" depends-on="clusterManager flowController">
+    <bean id="authorityProvider" class="org.apache.nifi.authorization.AuthorityProviderFactoryBean" depends-on="clusterManager">
         <property name="properties" ref="nifiProperties"/>
     </bean>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
index 2015530..4c48d77 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
@@ -16,12 +16,11 @@
  */
 package org.apache.nifi.cluster.manager.impl;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
+import org.apache.nifi.action.Action;
+import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.history.History;
 import org.apache.nifi.provenance.ProvenanceEventBuilder;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
@@ -31,12 +30,18 @@ import org.apache.nifi.provenance.search.QuerySubmission;
 import org.apache.nifi.provenance.search.SearchableField;
 import org.apache.nifi.reporting.EventAccess;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 public class ClusteredEventAccess implements EventAccess {
 
     private final WebClusterManager clusterManager;
+    private final AuditService auditService;
 
-    public ClusteredEventAccess(final WebClusterManager clusterManager) {
+    public ClusteredEventAccess(final WebClusterManager clusterManager, final AuditService auditService) {
         this.clusterManager = clusterManager;
+        this.auditService = auditService;
     }
 
     @Override
@@ -132,4 +137,10 @@ public class ClusteredEventAccess implements EventAccess {
             }
         };
     }
+
+    @Override
+    public List<Action> getFlowChanges(int firstActionId, int maxActions) {
+        final History history = auditService.getActions(firstActionId, maxActions);
+        return new ArrayList<>(history.getActions());
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index bfeec7a..2eabe0c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -1075,7 +1075,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
 
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this);
         final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler,
-                new ClusteredEventAccess(this), bulletinRepository, controllerServiceProvider, validationContextFactory);
+                new ClusteredEventAccess(this, auditService), bulletinRepository, controllerServiceProvider, validationContextFactory);
         taskNode.setName(task.getClass().getSimpleName());
 
         reportingTasks.put(id, taskNode);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 3f815b0..135025e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -16,39 +16,10 @@
  */
 package org.apache.nifi.controller;
 
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import javax.net.ssl.SSLContext;
-
+import com.sun.jersey.api.client.ClientHandlerException;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.action.Action;
+import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
@@ -139,6 +110,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
 import org.apache.nifi.groups.StandardProcessGroup;
+import org.apache.nifi.history.History;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.logging.ControllerServiceLogObserver;
 import org.apache.nifi.logging.LogLevel;
@@ -202,7 +174,36 @@ import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.jersey.api.client.ClientHandlerException;
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static java.util.Objects.requireNonNull;
 
 public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, Heartbeater, QueueProvider {
 
@@ -243,6 +244,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private final AtomicBoolean initialized = new AtomicBoolean(false);
     private final ControllerServiceProvider controllerServiceProvider;
     private final UserService userService;
+    private final AuditService auditService;
     private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
     private final ComponentStatusRepository componentStatusRepository;
     private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started
@@ -342,32 +344,36 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private static final Logger heartbeatLogger = LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat");
 
     public static FlowController createStandaloneInstance(
-        final FlowFileEventRepository flowFileEventRepo,
-        final NiFiProperties properties,
-        final UserService userService,
-        final StringEncryptor encryptor) {
+            final FlowFileEventRepository flowFileEventRepo,
+            final NiFiProperties properties,
+            final UserService userService,
+            final AuditService auditService,
+            final StringEncryptor encryptor) {
         return new FlowController(
-            flowFileEventRepo,
-            properties,
-            userService,
-            encryptor,
-            /* configuredForClustering */ false,
-            /* NodeProtocolSender */ null);
+                flowFileEventRepo,
+                properties,
+                userService,
+                auditService,
+                encryptor,
+                /* configuredForClustering */ false,
+                /* NodeProtocolSender */ null);
     }
 
     public static FlowController createClusteredInstance(
-        final FlowFileEventRepository flowFileEventRepo,
-        final NiFiProperties properties,
-        final UserService userService,
-        final StringEncryptor encryptor,
-        final NodeProtocolSender protocolSender) {
+            final FlowFileEventRepository flowFileEventRepo,
+            final NiFiProperties properties,
+            final UserService userService,
+            final AuditService auditService,
+            final StringEncryptor encryptor,
+            final NodeProtocolSender protocolSender) {
         final FlowController flowController = new FlowController(
-            flowFileEventRepo,
-            properties,
-            userService,
-            encryptor,
-            /* configuredForClustering */ true,
-            /* NodeProtocolSender */ protocolSender);
+                flowFileEventRepo,
+                properties,
+                userService,
+                auditService,
+                encryptor,
+                /* configuredForClustering */ true,
+                /* NodeProtocolSender */ protocolSender);
 
         flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
 
@@ -375,12 +381,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     private FlowController(
-        final FlowFileEventRepository flowFileEventRepo,
-        final NiFiProperties properties,
-        final UserService userService,
-        final StringEncryptor encryptor,
-        final boolean configuredForClustering,
-        final NodeProtocolSender protocolSender) {
+            final FlowFileEventRepository flowFileEventRepo,
+            final NiFiProperties properties,
+            final UserService userService,
+            final AuditService auditService,
+            final StringEncryptor encryptor,
+            final boolean configuredForClustering,
+            final NodeProtocolSender protocolSender) {
 
         maxTimerDrivenThreads = new AtomicInteger(10);
         maxEventDrivenThreads = new AtomicInteger(5);
@@ -428,6 +435,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         startConnectablesAfterInitialization = new ArrayList<>();
         startRemoteGroupPortsAfterInitialization = new ArrayList<>();
         this.userService = userService;
+        this.auditService = auditService;
 
         final String gracefulShutdownSecondsVal = properties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
         long shutdownSecs;
@@ -3638,7 +3646,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
     @Override
     public List<ProvenanceEventRecord> getProvenanceEvents(final long firstEventId, final int maxRecords) throws IOException {
-        return new ArrayList<ProvenanceEventRecord>(provenanceEventRepository.getEvents(firstEventId, maxRecords));
+        return new ArrayList<>(provenanceEventRepository.getEvents(firstEventId, maxRecords));
+    }
+
+    @Override
+    public List<Action> getFlowChanges(final int firstActionId, final int maxActions) {
+        final History history = auditService.getActions(firstActionId, maxActions);
+        return new ArrayList<>(history.getActions());
     }
 
     public void setClusterManagerRemoteSiteInfo(final Integer managerListeningPort, final Boolean commsSecure) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
index 6949cf9..c6c18c3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.spring;
 
+import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.cluster.protocol.NodeProtocolSender;
 import org.apache.nifi.controller.FlowController;
@@ -38,6 +39,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
     private FlowController flowController;
     private NiFiProperties properties;
     private UserService userService;
+    private AuditService auditService;
     private StringEncryptor encryptor;
 
     @Override
@@ -57,6 +59,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                         flowFileEventRepository,
                         properties,
                         userService,
+                        auditService,
                         encryptor,
                         nodeProtocolSender);
             } else {
@@ -64,6 +67,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                         flowFileEventRepository,
                         properties,
                         userService,
+                        auditService,
                         encryptor);
             }
 
@@ -98,4 +102,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
     public void setEncryptor(final StringEncryptor encryptor) {
         this.encryptor = encryptor;
     }
+
+    public void setAuditService(final AuditService auditService) {
+        this.auditService = auditService;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
index b0cdb11..e841b24 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
@@ -37,6 +37,7 @@
     <bean id="flowController" class="org.apache.nifi.spring.FlowControllerFactoryBean">
         <property name="properties" ref="nifiProperties"/>
         <property name="userService" ref="userService" />
+        <property name="auditService" ref="auditService" />
         <property name="encryptor" ref="stringEncryptor" />
     </bean>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index 7dc44da..b01b26c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
@@ -55,6 +56,7 @@ public class StandardFlowServiceTest {
     private NiFiProperties properties;
     private FlowFileEventRepository mockFlowFileEventRepository;
     private UserService mockUserService;
+    private AuditService mockAuditService;
     private StringEncryptor mockEncryptor;
 
     @BeforeClass
@@ -67,7 +69,8 @@ public class StandardFlowServiceTest {
         properties = NiFiProperties.getInstance();
         mockFlowFileEventRepository = mock(FlowFileEventRepository.class);
         mockUserService = mock(UserService.class);
-        flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, mockUserService, mockEncryptor);
+        mockAuditService = mock(AuditService.class);
+        flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, mockUserService, mockAuditService, mockEncryptor);
         flowService = StandardFlowService.createStandaloneInstance(flowController, properties, mockEncryptor);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
index 644826c..e8649ed 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml
@@ -21,4 +21,11 @@
         <version>0.3.1-SNAPSHOT</version>
     </parent>
     <artifactId>nifi-user-actions</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+    </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Action.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Action.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Action.java
deleted file mode 100644
index 689e852..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Action.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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.action;
-
-import java.io.Serializable;
-import org.apache.nifi.action.component.details.ComponentDetails;
-import org.apache.nifi.action.details.ActionDetails;
-import java.util.Date;
-
-/**
- *
- */
-public class Action implements Serializable {
-
-    private Integer id;
-    private String userDn;
-    private String userName;
-    private Date timestamp;
-
-    private String sourceId;
-    private String sourceName;
-    private Component sourceType;
-    private ComponentDetails componentDetails;
-
-    private Operation operation;
-    private ActionDetails actionDetails;
-
-    public Integer getId() {
-        return id;
-    }
-
-    public void setId(Integer id) {
-        this.id = id;
-    }
-
-    public Date getTimestamp() {
-        return timestamp;
-    }
-
-    public void setTimestamp(Date timestamp) {
-        this.timestamp = timestamp;
-    }
-
-    public String getUserDn() {
-        return userDn;
-    }
-
-    public void setUserDn(String userDn) {
-        this.userDn = userDn;
-    }
-
-    public String getUserName() {
-        return userName;
-    }
-
-    public void setUserName(String userName) {
-        this.userName = userName;
-    }
-
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    public Component getSourceType() {
-        return sourceType;
-    }
-
-    public void setSourceType(Component sourceType) {
-        this.sourceType = sourceType;
-    }
-
-    public ComponentDetails getComponentDetails() {
-        return componentDetails;
-    }
-
-    public void setComponentDetails(ComponentDetails componentDetails) {
-        this.componentDetails = componentDetails;
-    }
-
-    public Operation getOperation() {
-        return operation;
-    }
-
-    public void setOperation(Operation operation) {
-        this.operation = operation;
-    }
-
-    public ActionDetails getActionDetails() {
-        return actionDetails;
-    }
-
-    public void setActionDetails(ActionDetails actionDetails) {
-        this.actionDetails = actionDetails;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
deleted file mode 100644
index dcd23ee..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.action;
-
-/**
- * Defines possible components for a given action.
- */
-public enum Component {
-
-    Controller,
-    Processor,
-    InputPort,
-    OutputPort,
-    ProcessGroup,
-    RemoteProcessGroup,
-    Funnel,
-    Connection,
-    ControllerService,
-    ReportingTask;
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/FlowChangeAction.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/FlowChangeAction.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/FlowChangeAction.java
new file mode 100644
index 0000000..ad22059
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/FlowChangeAction.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.action;
+
+import org.apache.nifi.action.component.details.ComponentDetails;
+import org.apache.nifi.action.details.ActionDetails;
+import java.util.Date;
+
+/**
+ *
+ */
+public class FlowChangeAction implements Action {
+
+    private Integer id;
+    private String userIdentity;
+    private String userName;
+    private Date timestamp;
+
+    private String sourceId;
+    private String sourceName;
+    private Component sourceType;
+    private ComponentDetails componentDetails;
+
+    private Operation operation;
+    private ActionDetails actionDetails;
+
+    @Override
+    public Integer getId() {
+        return id;
+    }
+
+    public void setId(Integer id) {
+        this.id = id;
+    }
+
+    @Override
+    public Date getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(Date timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    @Override
+    public String getUserIdentity() {
+        return userIdentity;
+    }
+
+    public void setUserIdentity(String userIdentity) {
+        this.userIdentity = userIdentity;
+    }
+
+    @Override
+    public String getUserName() {
+        return userName;
+    }
+
+    public void setUserName(String userName) {
+        this.userName = userName;
+    }
+
+    @Override
+    public String getSourceId() {
+        return sourceId;
+    }
+
+    public void setSourceId(String sourceId) {
+        this.sourceId = sourceId;
+    }
+
+    @Override
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public void setSourceName(String sourceName) {
+        this.sourceName = sourceName;
+    }
+
+    @Override
+    public Component getSourceType() {
+        return sourceType;
+    }
+
+    public void setSourceType(Component sourceType) {
+        this.sourceType = sourceType;
+    }
+
+    @Override
+    public ComponentDetails getComponentDetails() {
+        return componentDetails;
+    }
+
+    public void setComponentDetails(ComponentDetails componentDetails) {
+        this.componentDetails = componentDetails;
+    }
+
+    @Override
+    public Operation getOperation() {
+        return operation;
+    }
+
+    public void setOperation(Operation operation) {
+        this.operation = operation;
+    }
+
+    @Override
+    public ActionDetails getActionDetails() {
+        return actionDetails;
+    }
+
+    public void setActionDetails(ActionDetails actionDetails) {
+        this.actionDetails = actionDetails;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Operation.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Operation.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Operation.java
deleted file mode 100644
index d1bb43f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Operation.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.action;
-
-/**
- * Defines possible operations for a given action.
- */
-public enum Operation {
-
-    Add,
-    Remove,
-    Paste,
-    Configure,
-    Move,
-    Disconnect,
-    Connect,
-    Start,
-    Stop,
-    Enable,
-    Disable,
-    Batch,
-    Purge;
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
deleted file mode 100644
index c05fa37..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ComponentDetails.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.action.component.details;
-
-import java.io.Serializable;
-
-/**
- *
- */
-public abstract class ComponentDetails implements Serializable {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
deleted file mode 100644
index 4c1f91e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.action.component.details;
-
-/**
- *
- */
-public class ExtensionDetails extends ComponentDetails {
-
-    private String type;
-
-    public String getType() {
-        return type;
-    }
-
-    public void setType(String type) {
-        this.type = type;
-    }
-
-}


[08/50] [abbrv] nifi git commit: NIFI-673: Rebased from master; Added InputRequirement annotation, as it is now merged into master

Posted by ma...@apache.org.
NIFI-673: Rebased from master; Added InputRequirement annotation, as it is now merged into master


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

Branch: refs/heads/NIFI-730
Commit: 385bfbb2c635f74aca318617175cd57a8e26ecda
Parents: 4e38288
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 25 11:14:11 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:14:11 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/processors/standard/FetchSFTP.java  | 3 +++
 .../main/java/org/apache/nifi/processors/standard/ListSFTP.java   | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/385bfbb2/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
index b95d864..8379987 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
@@ -22,6 +22,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -35,6 +37,7 @@ import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.SFTPTransfer;
 
 // Note that we do not use @SupportsBatching annotation. This processor cannot support batching because it must ensure that session commits happen before remote files are deleted.
+@InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"sftp", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"})
 @CapabilityDescription("Fetches the content of a file from a remote SFTP server and overwrites the contents of an incoming FlowFile with the content of the remote file.")
 @SeeAlso({GetSFTP.class, PutSFTP.class, GetFTP.class, PutFTP.class})

http://git-wip-us.apache.org/repos/asf/nifi/blob/385bfbb2/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
index 7226263..609b693 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
@@ -20,6 +20,8 @@ package org.apache.nifi.processors.standard;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.TriggerSerially;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
@@ -32,6 +34,7 @@ import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.SFTPTransfer;
 
 @TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
 @Tags({"list", "sftp", "remote", "ingest", "source", "input", "files"})
 @CapabilityDescription("Performs a listing of the files residing on an SFTP server. For each file that is found on the remote server, a new FlowFile will be created with the filename attribute "
     + "set to the name of the file on the remote server. This can then be used in conjunction with FetchSFTP in order to fetch those files.")


[39/50] [abbrv] nifi git commit: NIFI-1080 fleshed out InvokeHttp unit tests

Posted by ma...@apache.org.
NIFI-1080 fleshed out InvokeHttp unit tests

Signed-off-by: Matt Gilman <ma...@gmail.com>


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

Branch: refs/heads/NIFI-730
Commit: cef7b6c736721858a8bf11b2c03b0a2b0a49b0df
Parents: 9c148da
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Fri Oct 30 14:29:16 2015 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Sat Oct 31 00:19:36 2015 -0400

----------------------------------------------------------------------
 .../nifi/processors/standard/InvokeHTTP.java    |  21 +-
 .../processors/standard/TestInvokeHTTP.java     | 617 +-------------
 .../processors/standard/TestInvokeHttpSSL.java  |  90 ++
 .../standard/util/TestInvokeHttpCommon.java     | 830 +++++++++++++++++++
 4 files changed, 972 insertions(+), 586 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/cef7b6c7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index 848652a..d827658 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -228,7 +228,7 @@ public final class InvokeHTTP extends AbstractProcessor {
         public static final PropertyDescriptor PROP_ATTRIBUTES_TO_SEND = new PropertyDescriptor.Builder()
             .name("Attributes to Send")
             .description("Regular expression that defines which attributes to send as HTTP headers in the request. "
-                + "If not defined, no attributes are sent as headers.")
+                    + "If not defined, no attributes are sent as headers.")
             .required(false)
             .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
             .build();
@@ -301,36 +301,37 @@ public final class InvokeHTTP extends AbstractProcessor {
         // this is a "hidden" property - it's configured using a dynamic user property
         public static final PropertyDescriptor PROP_TRUSTED_HOSTNAME = new PropertyDescriptor.Builder()
             .name("Trusted Hostname")
-            .description("Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted "
-                + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .dynamic(true)
+            .description("Bypass the normal truststore hostname verifier to allow the specified remote hostname as trusted. "
+                    + "Enabling this property has MITM security implications, use wisely. Will still accept other connections based "
+                    + "on the normal truststore hostname verifier. Only valid with SSL (HTTPS) connections.")
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .dynamic(true)
             .build();
 
         // relationships
         public static final Relationship REL_SUCCESS_REQ = new Relationship.Builder()
             .name("Original")
-            .description("Original FlowFile will be routed upon success (2xx status codes).")
+                .description("Original FlowFile will be routed upon success (2xx status codes).")
             .build();
 
         public static final Relationship REL_SUCCESS_RESP = new Relationship.Builder()
             .name("Response")
-            .description("Response FlowFile will be routed upon success (2xx status codes).")
+                .description("Response FlowFile will be routed upon success (2xx status codes).")
             .build();
 
         public static final Relationship REL_RETRY = new Relationship.Builder()
             .name("Retry")
-            .description("FlowFile will be routed on any status code that can be retried (5xx status codes).")
+                .description("FlowFile will be routed on any status code that can be retried (5xx status codes).")
             .build();
 
         public static final Relationship REL_NO_RETRY = new Relationship.Builder()
             .name("No Retry")
-            .description("FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).")
+                .description("FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).")
             .build();
 
         public static final Relationship REL_FAILURE = new Relationship.Builder()
             .name("Failure")
-            .description("FlowFile will be routed on any type of connection failure, timeout or general exception.")
+                .description("FlowFile will be routed on any type of connection failure, timeout or general exception.")
             .build();
 
         public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(

http://git-wip-us.apache.org/repos/asf/nifi/blob/cef7b6c7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index a4fd3d7..a26b2ed 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -16,51 +16,26 @@
  */
 package org.apache.nifi.processors.standard;
 
-import static org.apache.commons.codec.binary.Base64.encodeBase64;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
-import java.nio.charset.StandardCharsets;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
+import java.net.URL;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processors.standard.InvokeHTTP.Config;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.ProvenanceEventType;
-import org.apache.nifi.ssl.StandardSSLContextService;
+import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
 import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class TestInvokeHTTP {
-
-    private static Map<String, String> sslProperties;
-    private static TestServer server;
-    private static String url;
-
-    private TestRunner runner;
+public class TestInvokeHTTP extends TestInvokeHttpCommon {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
@@ -68,16 +43,12 @@ public class TestInvokeHTTP {
         // don't commit this with this property enabled, or any 'mvn test' will be really verbose
         // System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard", "debug");
 
-        // create the SSL properties, which basically store keystore / trustore information
-        // this is used by the StandardSSLContextService and the Jetty Server
-        sslProperties = createSslProperties();
-
         // create a Jetty server on a random port
         server = createServer();
         server.startServer();
 
         // this is the base url with the random port
-        url = server.getSecureUrl();
+        url = server.getUrl();
     }
 
     @AfterClass
@@ -88,10 +59,6 @@ public class TestInvokeHTTP {
     @Before
     public void before() throws Exception {
         runner = TestRunners.newTestRunner(InvokeHTTP.class);
-        final StandardSSLContextService sslService = new StandardSSLContextService();
-        runner.addControllerService("ssl-context", sslService, sslProperties);
-        runner.enableControllerService(sslService);
-        runner.setProperty(Config.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
 
         server.clearHandlers();
     }
@@ -101,572 +68,70 @@ public class TestInvokeHTTP {
         runner.shutdown();
     }
 
-    private void addHandler(Handler handler) {
-        server.addHandler(handler);
-    }
-
-    @Test
-    public void testDateGeneration() throws Exception {
-        final DateHandler dh = new DateHandler();
-        addHandler(dh);
-
-        runner.setProperty(Config.PROP_URL, url);
-        createFlowFiles(runner);
-        runner.run();
-
-        // extract the date string sent to the server
-        // and store it as a java.util.Date
-        final SimpleDateFormat sdf = new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss z", Locale.US);
-        final Date date = sdf.parse(dh.dateString);
-
-        // calculate the difference between the date string sent by the client and
-        // the current system time -- these should be within a second or two
-        // (just enough time to run the test).
-        //
-        // If the difference is more like in hours, it's likely that a timezone
-        // conversion caused a problem.
-        final long diff = Math.abs(System.currentTimeMillis() - date.getTime());
-        final long threshold = 15000; // 15 seconds
-        if (diff > threshold) {
-            fail("Difference (" + diff + ") was greater than threshold (" + threshold + ")");
-        }
-        System.out.println("diff: " + diff);
-    }
-
-    @Test
-    public void test200() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/200");
-
-        createFlowFiles(runner);
-
-        runner.run();
-
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 1);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        // expected in request status.code and status.message
-        // original flow file (+attributes)??????????
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-        // expected in response
-        // status code, status message, all headers from server response --> ff attributes
-        // server response message body into payload of ff
-        // should not contain any original ff attributes
-        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
-        bundle1.assertContentEquals("/status/200".getBytes("UTF-8"));
-        bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle1.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        bundle1.assertAttributeEquals("Foo", "Bar");
-        bundle1.assertAttributeEquals("Content-Type", "text/plain; charset=ISO-8859-1");
-        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
-        final String expected1 = "/status/200";
-        Assert.assertEquals(expected1, actual1);
-
+    private static TestServer createServer() throws IOException {
+        return new TestServer();
     }
 
+    // Currently InvokeHttp does not support Proxy via Https
     @Test
-    public void test200auth() throws Exception {
-        addHandler(new BasicAuthHandler());
+    public void testProxy() throws Exception {
+        addHandler(new MyProxyHandler());
+        URL proxyURL = new URL(url);
 
-        final String username = "basic_user";
-        final String password = "basic_password";
-
-        runner.setProperty(Config.PROP_URL, url + "/status/200");
-        runner.setProperty(Config.PROP_BASIC_AUTH_USERNAME, username);
-        runner.setProperty(Config.PROP_BASIC_AUTH_PASSWORD, password);
-        final byte[] creds = String.format("%s:%s", username, password).getBytes(StandardCharsets.UTF_8);
-        final String expAuth = String.format("Basic %s", new String(encodeBase64(creds)));
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, "http://nifi.apache.org/"); // just a dummy URL no connection goes out
+        runner.setProperty(InvokeHTTP.Config.PROP_PROXY_HOST, proxyURL.getHost());
+        runner.setProperty(InvokeHTTP.Config.PROP_PROXY_PORT, String.valueOf(proxyURL.getPort()));
 
         createFlowFiles(runner);
 
         runner.run();
 
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 1);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        // expected in request status.code and status.message
-        // original flow file (+attributes)??????????
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        //expected in request status.code and status.message
+        //original flow file (+attributes)
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
         bundle.assertAttributeEquals("Foo", "Bar");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
 
-        // expected in response
-        // status code, status message, all headers from server response --> ff attributes
-        // server response message body into payload of ff
-        // should not contain any original ff attributes
-        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
-        final String bundle1Content = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
-        assertTrue(bundle1Content.startsWith(expAuth)); // use startsWith instead of equals so we can ignore line endings
-        bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle1.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
+        //expected in response
+        //status code, status message, all headers from server response --> ff attributes
+        //server response message body into payload of ff
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("http://nifi.apache.org/".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
         bundle1.assertAttributeEquals("Foo", "Bar");
         bundle1.assertAttributeEquals("Content-Type", "text/plain; charset=ISO-8859-1");
-
-        final List<ProvenanceEventRecord> provEvents = runner.getProvenanceEvents();
-        assertEquals(2, provEvents.size());
-        boolean forkEvent = false;
-        boolean fetchEvent = false;
-        for (final ProvenanceEventRecord event : provEvents) {
-            if (event.getEventType() == ProvenanceEventType.FORK) {
-                forkEvent = true;
-            } else if (event.getEventType() == ProvenanceEventType.FETCH) {
-                fetchEvent = true;
-            }
-        }
-
-        assertTrue(forkEvent);
-        assertTrue(fetchEvent);
-    }
-
-    @Test
-    public void test401notauth() throws Exception {
-        addHandler(new BasicAuthHandler());
-
-        final String username = "basic_user";
-        final String password = "basic_password";
-
-        runner.setProperty(Config.PROP_URL, url + "/status/401");
-        runner.setProperty(Config.PROP_BASIC_AUTH_USERNAME, username);
-        runner.setProperty(Config.PROP_BASIC_AUTH_PASSWORD, password);
-
-        createFlowFiles(runner);
-
-        runner.run();
-
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 1);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        // expected in request status.code and status.message
-        // original flow file (+attributes)??????????
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "401");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Unauthorized");
-        bundle.assertAttributeEquals("Foo", "Bar");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-
-        final String response = bundle.getAttribute(Config.RESPONSE_BODY);
-        assertEquals(response, "Get off my lawn!");
-    }
-
-    @Test
-    public void test500() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/500");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 1);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        // expected in response
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_RETRY).get(0);
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "500");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Server Error");
-        bundle.assertAttributeEquals(Config.RESPONSE_BODY, "/status/500");
-
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
     }
 
-    @Test
-    public void test300() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/302");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 1);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-        // getMyFlowFiles();
-        // expected in response
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "302");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Found");
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-    }
-
-    @Test
-    public void test304() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/304");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 1);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-        // getMyFlowFiles();
-        // expected in response
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "304");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Not Modified");
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-    }
-
-    @Test
-    public void test400() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/400");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 1);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-        // getMyFlowFiles();
-        // expected in response
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "400");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Bad Request");
-        bundle.assertAttributeEquals(Config.RESPONSE_BODY, "/status/400");
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-    }
-
-    @Test
-    public void test412() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_URL, url + "/status/412");
-        runner.setProperty(Config.PROP_METHOD, "GET");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 1);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        // expected in response
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "412");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Precondition Failed");
-        bundle.assertAttributeEquals(Config.RESPONSE_BODY, "/status/412");
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-    }
-
-    @Test
-    public void testHead() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        runner.setProperty(Config.PROP_METHOD, "HEAD");
-        runner.setProperty(Config.PROP_URL, url + "/status/200");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 1);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
-        bundle1.assertContentEquals("".getBytes("UTF-8"));
-        bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle1.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        bundle1.assertAttributeEquals("Foo", "Bar");
-        bundle1.assertAttributeEquals("Content-Type", "text/plain");
-        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
-        final String expected1 = "";
-        Assert.assertEquals(expected1, actual1);
-    }
-
-    @Test
-    public void testPost() throws Exception {
-        addHandler(new PostHandler());
-
-        runner.setProperty(Config.PROP_METHOD, "POST");
-        runner.setProperty(Config.PROP_URL, url + "/post");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 1);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
-        bundle1.assertContentEquals("".getBytes("UTF-8"));
-        bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle1.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        bundle1.assertAttributeEquals("Foo", "Bar");
-        bundle1.assertAttributeNotExists("Content-Type");
-
-        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
-        final String expected1 = "";
-        Assert.assertEquals(expected1, actual1);
-    }
-
-    @Test
-    public void testPut() throws Exception {
-        addHandler(new PostHandler());
-
-        runner.setProperty(Config.PROP_METHOD, "PUT");
-        runner.setProperty(Config.PROP_URL, url + "/post");
-
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 1);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 0);
-
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
-        bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-
-        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
-        bundle1.assertContentEquals("".getBytes("UTF-8"));
-        bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
-        bundle1.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
-        bundle1.assertAttributeEquals("Foo", "Bar");
-        bundle1.assertAttributeNotExists("Content-Type");
-
-        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
-        final String expected1 = "";
-        Assert.assertEquals(expected1, actual1);
-    }
-
-    @Test
-    public void testConnectFailBadPort() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        // this is the bad urls
-        final String badurlport = "https://localhost:" + 445;
-
-        runner.setProperty(Config.PROP_URL, badurlport + "/doesnotExist");
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 1);
-
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_FAILURE).get(0);
-
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-    }
-
-    @Test
-    public void testConnectFailBadHost() throws Exception {
-        addHandler(new GetOrHeadHandler());
-
-        final String badurlhost = "https://localhOOst:" + 445;
-
-        runner.setProperty(Config.PROP_URL, badurlhost + "/doesnotExist");
-        createFlowFiles(runner);
-
-        runner.run();
-        runner.assertTransferCount(Config.REL_SUCCESS_REQ, 0);
-        runner.assertTransferCount(Config.REL_SUCCESS_RESP, 0);
-        runner.assertTransferCount(Config.REL_RETRY, 0);
-        runner.assertTransferCount(Config.REL_NO_RETRY, 0);
-        runner.assertTransferCount(Config.REL_FAILURE, 1);
-
-        final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_FAILURE).get(0);
-
-        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
-        final String expected = "Hello";
-        Assert.assertEquals(expected, actual);
-        bundle.assertAttributeEquals("Foo", "Bar");
-    }
-
-    private static Map<String, String> createSslProperties() {
-        final Map<String, String> map = new HashMap<>();
-        map.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-        map.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-        map.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
-        map.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-        map.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
-        map.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
-        return map;
-    }
-
-    private static TestServer createServer() throws IOException {
-        return new TestServer(sslProperties);
-    }
-
-    private static void createFlowFiles(final TestRunner testRunner) throws UnsupportedEncodingException {
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
-        attributes.put("Foo", "Bar");
-        testRunner.enqueue("Hello".getBytes("UTF-8"), attributes);
-
-    }
-
-    private static class PostHandler extends AbstractHandler {
-
-        @Override
-        public void handle(String target, Request baseRequest,
-            HttpServletRequest request, HttpServletResponse response)
-                throws IOException, ServletException {
-
-            baseRequest.setHandled(true);
-
-            assertEquals("/post", target);
-
-            final String body = request.getReader().readLine();
-            assertEquals("Hello", body);
-
-        }
-    }
-
-    private static class GetOrHeadHandler extends AbstractHandler {
+    public static class MyProxyHandler extends AbstractHandler {
 
         @Override
         public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
             baseRequest.setHandled(true);
 
-            final int status = Integer.valueOf(target.substring("/status".length() + 1));
-            response.setStatus(status);
-
-            response.setContentType("text/plain");
-            response.setContentLength(target.length());
+            if ("Get".equalsIgnoreCase(request.getMethod())) {
+                response.setStatus(200);
+                String proxyPath = baseRequest.getUri().toString();
+                response.setContentLength(proxyPath.length());
+                response.setContentType("text/plain");
 
-            if ("GET".equalsIgnoreCase(request.getMethod())) {
                 try (PrintWriter writer = response.getWriter()) {
-                    writer.print(target);
+                    writer.print(proxyPath);
                     writer.flush();
                 }
-            }
-
-        }
-
-    }
-
-    private static class DateHandler extends AbstractHandler {
-
-        private String dateString;
-
-        @Override
-        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
-            baseRequest.setHandled(true);
-
-            dateString = request.getHeader("Date");
-
-            response.setStatus(200);
-            response.setContentType("text/plain");
-            response.getWriter().println("Way to go!");
-        }
-    }
-
-    private static class BasicAuthHandler extends AbstractHandler {
-
-        private String authString;
-
-        @Override
-        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
-            baseRequest.setHandled(true);
-
-            authString = request.getHeader("Authorization");
-
-            final int status = Integer.valueOf(target.substring("/status".length() + 1));
-
-            if (status == 200) {
-                response.setStatus(status);
-                response.setContentType("text/plain");
-                response.getWriter().println(authString);
             } else {
-                response.setStatus(status);
+                response.setStatus(404);
                 response.setContentType("text/plain");
-                response.getWriter().println("Get off my lawn!");
+                response.setContentLength(0);
             }
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/cef7b6c7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
new file mode 100644
index 0000000..d155b74
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import org.apache.nifi.processors.standard.InvokeHTTP.Config;
+import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
+import org.apache.nifi.ssl.StandardSSLContextService;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestInvokeHttpSSL extends TestInvokeHttpCommon {
+
+    private static Map<String, String> sslProperties;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        // useful for verbose logging output
+        // don't commit this with this property enabled, or any 'mvn test' will be really verbose
+        // System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard", "debug");
+
+        // create the SSL properties, which basically store keystore / trustore information
+        // this is used by the StandardSSLContextService and the Jetty Server
+        sslProperties = createSslProperties();
+
+        // create a Jetty server on a random port
+        server = createServer();
+        server.startServer();
+
+        // this is the base url with the random port
+        url = server.getSecureUrl();
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        server.shutdownServer();
+    }
+
+    @Before
+    public void before() throws Exception {
+        runner = TestRunners.newTestRunner(InvokeHTTP.class);
+        final StandardSSLContextService sslService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslService, sslProperties);
+        runner.enableControllerService(sslService);
+        runner.setProperty(Config.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
+
+        server.clearHandlers();
+    }
+
+    @After
+    public void after() {
+        runner.shutdown();
+    }
+
+    private static TestServer createServer() throws IOException {
+        return new TestServer(sslProperties);
+    }
+
+    private static Map<String, String> createSslProperties() {
+        final Map<String, String> map = new HashMap<>();
+        map.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        map.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        map.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        map.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        map.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        map.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        return map;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/cef7b6c7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
new file mode 100644
index 0000000..88dfcdb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
@@ -0,0 +1,830 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard.util;
+
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processors.standard.InvokeHTTP;
+import org.apache.nifi.processors.standard.TestServer;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.junit.Assert;
+import org.junit.Test;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.StandardCharsets;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.commons.codec.binary.Base64.encodeBase64;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public abstract class TestInvokeHttpCommon {
+
+    public static TestServer server;
+    public static String url;
+
+    public TestRunner runner;
+
+
+    public void addHandler(Handler handler) {
+        server.addHandler(handler);
+    }
+
+    @Test
+    public void testDateGeneration() throws Exception {
+
+        final DateHandler dh = new DateHandler();
+        addHandler(dh);
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url);
+        createFlowFiles(runner);
+        runner.run();
+
+        // extract the date string sent to the server
+        // and store it as a java.util.Date
+        final SimpleDateFormat sdf = new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss z", Locale.US);
+        final Date date = sdf.parse(dh.dateString);
+
+        // calculate the difference between the date string sent by the client and
+        // the current system time -- these should be within a second or two
+        // (just enough time to run the test).
+        //
+        // If the difference is more like in hours, it's likely that a timezone
+        // conversion caused a problem.
+        final long diff = Math.abs(System.currentTimeMillis() - date.getTime());
+        final long threshold = 15000; // 15 seconds
+        if (diff > threshold) {
+            fail("Difference (" + diff + ") was greater than threshold (" + threshold + ")");
+        }
+        System.out.println("diff: " + diff);
+    }
+
+    @Test
+    public void test200() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+
+        createFlowFiles(runner);
+
+        runner.run();
+
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        // expected in request status.code and status.message
+        // original flow file (+attributes)
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        // expected in response
+        // status code, status message, all headers from server response --> ff attributes
+        // server response message body into payload of ff
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("/status/200".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeEquals("Content-Type", "text/plain; charset=ISO-8859-1");
+    }
+
+    @Test
+    public void test200Auth() throws Exception {
+        addHandler(new BasicAuthHandler());
+
+        final String username = "basic_user";
+        final String password = "basic_password";
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+        runner.setProperty(InvokeHTTP.Config.PROP_BASIC_AUTH_USERNAME, username);
+        runner.setProperty(InvokeHTTP.Config.PROP_BASIC_AUTH_PASSWORD, password);
+        final byte[] creds = String.format("%s:%s", username, password).getBytes(StandardCharsets.UTF_8);
+        final String expAuth = String.format("Basic %s", new String(encodeBase64(creds)));
+
+        createFlowFiles(runner);
+
+        runner.run();
+
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        // expected in request status.code and status.message
+        // original flow file (+attributes)
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        // expected in response
+        // status code, status message, all headers from server response --> ff attributes
+        // server response message body into payload of ff
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        final String bundle1Content = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
+        assertTrue(bundle1Content.startsWith(expAuth)); // use startsWith instead of equals so we can ignore line endings
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeEquals("Content-Type", "text/plain; charset=ISO-8859-1");
+
+        final List<ProvenanceEventRecord> provEvents = runner.getProvenanceEvents();
+        assertEquals(2, provEvents.size());
+        boolean forkEvent = false;
+        boolean fetchEvent = false;
+        for (final ProvenanceEventRecord event : provEvents) {
+            if (event.getEventType() == ProvenanceEventType.FORK) {
+                forkEvent = true;
+            } else if (event.getEventType() == ProvenanceEventType.FETCH) {
+                fetchEvent = true;
+            }
+        }
+
+        assertTrue(forkEvent);
+        assertTrue(fetchEvent);
+    }
+
+    @Test
+    public void test401NotAuth() throws Exception {
+        addHandler(new BasicAuthHandler());
+
+        final String username = "basic_user";
+        final String password = "basic_password";
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/401");
+        runner.setProperty(InvokeHTTP.Config.PROP_BASIC_AUTH_USERNAME, username);
+        runner.setProperty(InvokeHTTP.Config.PROP_BASIC_AUTH_PASSWORD, password);
+
+        createFlowFiles(runner);
+
+        runner.run();
+
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        // expected in request status.code and status.message
+        // original flow file (+attributes)
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_NO_RETRY).get(0);
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "401");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Unauthorized");
+        bundle.assertAttributeEquals("Foo", "Bar");
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+
+        final String response = bundle.getAttribute(InvokeHTTP.Config.RESPONSE_BODY);
+        assertEquals(response, "Get off my lawn!");
+    }
+
+    @Test
+    public void test500() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/500");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        // expected in response
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_RETRY).get(0);
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "500");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Server Error");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.RESPONSE_BODY, "/status/500");
+
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+    }
+
+    @Test
+    public void test300() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/302");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+        // getMyFlowFiles();
+        // expected in response
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_NO_RETRY).get(0);
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "302");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Found");
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+    }
+
+    @Test
+    public void test304() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/304");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+        // getMyFlowFiles();
+        // expected in response
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_NO_RETRY).get(0);
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "304");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Not Modified");
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+    }
+
+    @Test
+    public void test400() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/400");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+        // getMyFlowFiles();
+        // expected in response
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_NO_RETRY).get(0);
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "400");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Bad Request");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.RESPONSE_BODY, "/status/400");
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+    }
+
+    @Test
+    public void test412() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/412");
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "GET");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        // expected in response
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_NO_RETRY).get(0);
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "412");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "Precondition Failed");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.RESPONSE_BODY, "/status/412");
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+    }
+
+    @Test
+    public void testHead() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "HEAD");
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeEquals("Content-Type", "text/plain");
+        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
+        final String expected1 = "";
+        Assert.assertEquals(expected1, actual1);
+    }
+
+    @Test
+    public void testPost() throws Exception {
+        addHandler(new PostHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "POST");
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/post");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeNotExists("Content-Type");
+
+        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
+        final String expected1 = "";
+        Assert.assertEquals(expected1, actual1);
+    }
+
+    @Test
+    public void testPut() throws Exception {
+        addHandler(new PostHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "PUT");
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/post");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeNotExists("Content-Type");
+
+        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
+        final String expected1 = "";
+        Assert.assertEquals(expected1, actual1);
+    }
+
+    @Test
+    public void testDelete() throws Exception {
+        addHandler(new DeleteHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "DELETE");
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        final String actual1 = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
+        final String expected1 = "";
+        Assert.assertEquals(expected1, actual1);
+    }
+
+    @Test
+    public void testOptions() throws Exception {
+        addHandler(new OptionsHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_METHOD, "OPTIONS");
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("/status/200".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+    }
+
+    @Test
+    public void testSendAttributes() throws Exception {
+        addHandler(new AttributesSentHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+        runner.setProperty(InvokeHTTP.Config.PROP_ATTRIBUTES_TO_SEND, "Foo");
+
+        createFlowFiles(runner);
+
+        runner.run();
+
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 1);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 0);
+
+        //expected in request status.code and status.message
+        //original flow file (+attributes)
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_REQ).get(0);
+        bundle.assertContentEquals("Hello".getBytes("UTF-8"));
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle.assertAttributeEquals("Foo", "Bar");
+
+        //expected in response
+        //status code, status message, all headers from server response --> ff attributes
+        //server response message body into payload of ff
+        final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_SUCCESS_RESP).get(0);
+        bundle1.assertContentEquals("Bar".getBytes("UTF-8"));
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_CODE, "200");
+        bundle1.assertAttributeEquals(InvokeHTTP.Config.STATUS_MESSAGE, "OK");
+        bundle1.assertAttributeEquals("Foo", "Bar");
+        bundle1.assertAttributeEquals("Content-Type", "text/plain; charset=ISO-8859-1");
+    }
+
+    @Test
+    public void testReadTimeout() throws Exception {
+        addHandler(new ReadTimeoutHandler());
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, url + "/status/200");
+        runner.setProperty(InvokeHTTP.Config.PROP_READ_TIMEOUT, "5 secs");
+
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 1);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_FAILURE).get(0);
+
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+    }
+
+    @Test
+    public void testConnectFailBadPort() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        // this is the bad urls
+        final String badurlport = "http://localhost:" + 445;
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, badurlport + "/doesnotExist");
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 1);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_FAILURE).get(0);
+
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+    }
+
+    @Test
+    public void testConnectFailBadHost() throws Exception {
+        addHandler(new GetOrHeadHandler());
+
+        final String badurlhost = "http://localhOOst:" + 445;
+
+        runner.setProperty(InvokeHTTP.Config.PROP_URL, badurlhost + "/doesnotExist");
+        createFlowFiles(runner);
+
+        runner.run();
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_REQ, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_SUCCESS_RESP, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.Config.REL_FAILURE, 1);
+
+        final MockFlowFile bundle = runner.getFlowFilesForRelationship(InvokeHTTP.Config.REL_FAILURE).get(0);
+
+        final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
+        final String expected = "Hello";
+        Assert.assertEquals(expected, actual);
+        bundle.assertAttributeEquals("Foo", "Bar");
+    }
+
+
+    public static void createFlowFiles(final TestRunner testRunner) throws UnsupportedEncodingException {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(CoreAttributes.MIME_TYPE.key(), "application/plain-text");
+        attributes.put("Foo", "Bar");
+        testRunner.enqueue("Hello".getBytes("UTF-8"), attributes);
+
+    }
+
+
+    private static class DateHandler extends AbstractHandler {
+
+        private String dateString;
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            dateString = request.getHeader("Date");
+
+            response.setStatus(200);
+            response.setContentType("text/plain");
+            response.getWriter().println("Way to go!");
+        }
+    }
+
+    private static class PostHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest,
+                           HttpServletRequest request, HttpServletResponse response)
+                throws IOException, ServletException {
+
+            baseRequest.setHandled(true);
+
+            assertEquals("/post", target);
+
+            final String body = request.getReader().readLine();
+            assertEquals("Hello", body);
+
+        }
+    }
+
+    private static class GetOrHeadHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            final int status = Integer.valueOf(target.substring("/status".length() + 1));
+            response.setStatus(status);
+
+            response.setContentType("text/plain");
+            response.setContentLength(target.length());
+
+            if ("GET".equalsIgnoreCase(request.getMethod())) {
+                try (PrintWriter writer = response.getWriter()) {
+                    writer.print(target);
+                    writer.flush();
+                }
+            }
+
+        }
+    }
+
+    private static class DeleteHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            if ("DELETE".equalsIgnoreCase(request.getMethod())) {
+                final int status = Integer.valueOf(target.substring("/status".length() + 1));
+                response.setStatus(status);
+                response.setContentLength(0);
+            } else {
+                response.setStatus(404);
+                response.setContentType("text/plain");
+                response.setContentLength(0);
+            }
+        }
+    }
+
+    private static class OptionsHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            if ("OPTIONS".equalsIgnoreCase(request.getMethod())) {
+                final int status = Integer.valueOf(target.substring("/status".length() + 1));
+                response.setStatus(status);
+                response.setContentLength(target.length());
+                response.setContentType("text/plain");
+
+                try (PrintWriter writer = response.getWriter()) {
+                    writer.print(target);
+                    writer.flush();
+                }
+            } else {
+                response.setStatus(404);
+                response.setContentType("text/plain");
+                response.setContentLength(target.length());
+            }
+        }
+    }
+
+    private static class AttributesSentHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            if ("Get".equalsIgnoreCase(request.getMethod())) {
+                String headerValue = request.getHeader("Foo");
+                final int status = Integer.valueOf(target.substring("/status".length() + 1));
+                response.setStatus(status);
+                response.setContentLength(headerValue.length());
+                response.setContentType("text/plain");
+
+
+                try (PrintWriter writer = response.getWriter()) {
+                    writer.print(headerValue);
+                    writer.flush();
+                }
+            } else {
+                response.setStatus(404);
+                response.setContentType("text/plain");
+                response.setContentLength(0);
+            }
+        }
+    }
+
+    private static class ReadTimeoutHandler extends AbstractHandler {
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            if ("Get".equalsIgnoreCase(request.getMethod())) {
+                try {
+                    Thread.sleep(10000);
+                } catch (InterruptedException e) {
+                    return;
+                }
+                String headerValue = request.getHeader("Foo");
+                headerValue = headerValue == null ? "" : headerValue;
+                final int status = Integer.valueOf(target.substring("/status".length() + 1));
+                response.setStatus(status);
+                response.setContentLength(headerValue.length());
+                response.setContentType("text/plain");
+
+                try (PrintWriter writer = response.getWriter()) {
+                    writer.print(headerValue);
+                    writer.flush();
+                }
+            } else {
+                response.setStatus(404);
+                response.setContentType("text/plain");
+                response.setContentLength(0);
+            }
+        }
+    }
+
+
+
+    private static class BasicAuthHandler extends AbstractHandler {
+
+        private String authString;
+
+        @Override
+        public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+            baseRequest.setHandled(true);
+
+            authString = request.getHeader("Authorization");
+
+            if (authString == null) {
+                response.setStatus(401);
+                response.setHeader("WWW-Authenticate", "Basic realm=\"Jetty\"");
+                response.setHeader("response.phrase", "Unauthorized");
+                response.setContentType("text/plain");
+                response.getWriter().println("Get off my lawn!");
+                return;
+            }
+
+            final int status = Integer.valueOf(target.substring("/status".length() + 1));
+
+            if (status == 200) {
+                response.setStatus(status);
+                response.setContentType("text/plain");
+                response.getWriter().println(authString);
+            } else {
+                response.setStatus(status);
+                response.setContentType("text/plain");
+                response.getWriter().println("Get off my lawn!");
+            }
+        }
+    }
+
+}


[41/50] [abbrv] nifi git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/nifi

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


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

Branch: refs/heads/NIFI-730
Commit: b729bf4c196e0fbd33692f76c11931aef61c650b
Parents: 6e193df cef7b6c
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Nov 1 14:16:54 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Nov 1 14:16:54 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/InvokeHTTP.java    |  21 +-
 .../processors/standard/TestInvokeHTTP.java     | 617 +-------------
 .../processors/standard/TestInvokeHttpSSL.java  |  90 ++
 .../standard/util/TestInvokeHttpCommon.java     | 830 +++++++++++++++++++
 4 files changed, 972 insertions(+), 586 deletions(-)
----------------------------------------------------------------------



[14/50] [abbrv] nifi git commit: NIFI-10: Added FETCH Provenance Event and updated processors to use this new event type

Posted by ma...@apache.org.
NIFI-10: Added FETCH Provenance Event and updated processors to use this new event type


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

Branch: refs/heads/NIFI-730
Commit: 51f564024a2fbe7fbd08760635561f08619be0e4
Parents: aec32a2
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Oct 15 17:00:20 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 14:58:50 2015 -0400

----------------------------------------------------------------------
 .../nifi/provenance/ProvenanceEventType.java    |  26 ++-
 .../nifi/provenance/ProvenanceReporter.java     |  37 ++++
 .../nifi/util/MockProvenanceReporter.java       |  35 ++-
 .../nifi/processors/aws/s3/FetchS3Object.java   |   2 +-
 .../repository/StandardProvenanceReporter.java  |  31 ++-
 .../nifi/processors/standard/InvokeHTTP.java    | 211 +++++++++----------
 .../processors/standard/TestInvokeHTTP.java     |  54 ++---
 7 files changed, 258 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
index e5e47b7..188e8fc 100644
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
+++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
@@ -23,45 +23,66 @@ public enum ProvenanceEventType {
      * not received from a remote system or external process
      */
     CREATE,
+
     /**
-     * Indicates a provenance event for receiving data from an external process
+     * Indicates a provenance event for receiving data from an external process. This Event Type
+     * is expected to be the first event for a FlowFile. As such, a Processor that receives data
+     * from an external source and uses that data to replace the content of an existing FlowFile
+     * should use the {@link #FETCH} event type, rather than the RECEIVE event type.
      */
     RECEIVE,
+
+    /**
+     * Indicates that the contents of a FlowFile were overwritten using the contents of some
+     * external resource. This is similar to the {@link #RECEIVE} event but varies in that
+     * RECEIVE events are intended to be used as the event that introduces the FlowFile into
+     * the system, whereas FETCH is used to indicate that the contents of an existing FlowFile
+     * were overwritten.
+     */
+    FETCH,
+
     /**
      * Indicates a provenance event for sending data to an external process
      */
     SEND,
+
     /**
      * Indicates a provenance event for the conclusion of an object's life for
      * some reason other than object expiration
      */
     DROP,
+
     /**
      * Indicates a provenance event for the conclusion of an object's life due
      * to the fact that the object could not be processed in a timely manner
      */
     EXPIRE,
+
     /**
      * FORK is used to indicate that one or more FlowFile was derived from a
      * parent FlowFile.
      */
     FORK,
+
     /**
      * JOIN is used to indicate that a single FlowFile is derived from joining
      * together multiple parent FlowFiles.
      */
     JOIN,
+
     /**
      * CLONE is used to indicate that a FlowFile is an exact duplicate of its
      * parent FlowFile.
      */
     CLONE,
+
     /**
      * CONTENT_MODIFIED is used to indicate that a FlowFile's content was
      * modified in some way. When using this Event Type, it is advisable to
      * provide details about how the content is modified.
      */
     CONTENT_MODIFIED,
+
     /**
      * ATTRIBUTES_MODIFIED is used to indicate that a FlowFile's attributes were
      * modified in some way. This event is not needed when another event is
@@ -69,17 +90,20 @@ public enum ProvenanceEventType {
      * FlowFile attributes.
      */
     ATTRIBUTES_MODIFIED,
+
     /**
      * ROUTE is used to show that a FlowFile was routed to a specified
      * {@link org.apache.nifi.processor.Relationship Relationship} and should provide
      * information about why the FlowFile was routed to this relationship.
      */
     ROUTE,
+
     /**
      * Indicates a provenance event for adding additional information such as a
      * new linkage to a new URI or UUID
      */
     ADDINFO,
+
     /**
      * Indicates a provenance event for replaying a FlowFile. The UUID of the
      * event will indicate the UUID of the original FlowFile that is being

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
index db589f8..0fd29fd 100644
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
+++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
@@ -124,6 +124,43 @@ public interface ProvenanceReporter {
     void receive(FlowFile flowFile, String transitUri, String sourceSystemFlowFileIdentifier, String details, long transmissionMillis);
 
     /**
+     * Emits a Provenance Event of type
+     * {@link ProvenanceEventType#FETCH FETCH} that indicates that the content of the given
+     * FlowFile was overwritten with the data received from an external source.
+     *
+     * @param flowFile the FlowFile whose content was replaced
+     * @param transitUri A URI that provides information about the System and
+     * Protocol information over which the transfer occurred.
+     */
+    void fetch(FlowFile flowFile, String transitUri);
+
+    /**
+     * Emits a Provenance Event of type
+     * {@link ProvenanceEventType#FETCH FETCH} that indicates that the content of the given
+     * FlowFile was overwritten with the data received from an external source.
+     *
+     * @param flowFile the FlowFile whose content was replaced
+     * @param transitUri A URI that provides information about the System and
+     * Protocol information over which the transfer occurred.
+     * @param transmissionMillis the number of milliseconds taken to transfer the data
+     */
+    void fetch(FlowFile flowFile, String transitUri, long transmissionMillis);
+
+    /**
+     * Emits a Provenance Event of type
+     * {@link ProvenanceEventType#FETCH FETCH} that indicates that the content of the given
+     * FlowFile was overwritten with the data received from an external source.
+     *
+     * @param flowFile the FlowFile whose content was replaced
+     * @param transitUri A URI that provides information about the System and
+     * Protocol information over which the transfer occurred.
+     * @param details details about the event
+     * @param transmissionMillis the number of milliseconds taken to transfer
+     * the data
+     */
+    void fetch(FlowFile flowFile, String transitUri, String details, long transmissionMillis);
+    
+    /**
      * Emits a Provenance Event of type {@link ProvenanceEventType#SEND SEND}
      * that indicates that a copy of the given FlowFile was sent to an external
      * destination. The external destination may be a remote system or may be a

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-mock/src/main/java/org/apache/nifi/util/MockProvenanceReporter.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProvenanceReporter.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProvenanceReporter.java
index 8c9a320..8458715 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProvenanceReporter.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProvenanceReporter.java
@@ -124,7 +124,40 @@ public class MockProvenanceReporter implements ProvenanceReporter {
 
         try {
             final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.RECEIVE)
-                .setTransitUri(transitUri).setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier).setEventDuration(transmissionMillis).setDetails(details).build();
+                .setTransitUri(transitUri)
+                .setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier)
+                .setEventDuration(transmissionMillis)
+                .setDetails(details)
+                .build();
+            events.add(record);
+        } catch (final Exception e) {
+            logger.error("Failed to generate Provenance Event due to " + e);
+            if (logger.isDebugEnabled()) {
+                logger.error("", e);
+            }
+        }
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri) {
+        fetch(flowFile, transitUri, -1L);
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri, final long transmissionMillis) {
+        fetch(flowFile, transitUri, null, transmissionMillis);
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri, final String details, final long transmissionMillis) {
+        verifyFlowFileKnown(flowFile);
+
+        try {
+            final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.FETCH)
+                .setTransitUri(transitUri)
+                .setEventDuration(transmissionMillis)
+                .setDetails(details)
+                .build();
             events.add(record);
         } catch (final Exception e) {
             logger.error("Failed to generate Provenance Event due to " + e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
index 131e671..bc6aeec 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java
@@ -152,7 +152,7 @@ public class FetchS3Object extends AbstractS3Processor {
         session.transfer(flowFile, REL_SUCCESS);
         final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
         getLogger().info("Successfully retrieved S3 Object for {} in {} millis; routing to success", new Object[]{flowFile, transferMillis});
-        session.getProvenanceReporter().receive(flowFile, "http://" + bucket + ".amazonaws.com/" + key, transferMillis);
+        session.getProvenanceReporter().fetch(flowFile, "http://" + bucket + ".amazonaws.com/" + key, transferMillis);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java
index 8852f42..8a89dbf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProvenanceReporter.java
@@ -124,7 +124,36 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
 
         try {
             final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.RECEIVE)
-                    .setTransitUri(transitUri).setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier).setEventDuration(transmissionMillis).setDetails(details).build();
+                .setTransitUri(transitUri).setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier).setEventDuration(transmissionMillis).setDetails(details).build();
+            events.add(record);
+        } catch (final Exception e) {
+            logger.error("Failed to generate Provenance Event due to " + e);
+            if (logger.isDebugEnabled()) {
+                logger.error("", e);
+            }
+        }
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri) {
+        fetch(flowFile, transitUri, -1L);
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri, final long transmissionMillis) {
+        fetch(flowFile, transitUri, null, transmissionMillis);
+    }
+
+    @Override
+    public void fetch(final FlowFile flowFile, final String transitUri, final String details, final long transmissionMillis) {
+        verifyFlowFileKnown(flowFile);
+
+        try {
+            final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.FETCH)
+                .setTransitUri(transitUri)
+                .setEventDuration(transmissionMillis)
+                .setDetails(details)
+                .build();
             events.add(record);
         } catch (final Exception e) {
             logger.error("Failed to generate Provenance Event due to " + e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index c7be728..848652a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -89,7 +89,7 @@ import org.joda.time.format.DateTimeFormatter;
     @WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"),
     @WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server")})
 @DynamicProperty(name = "Trusted Hostname", value = "A hostname", description = "Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted "
-        + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.")
+    + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.")
 public final class InvokeHTTP extends AbstractProcessor {
 
     @Override
@@ -170,76 +170,75 @@ public final class InvokeHTTP extends AbstractProcessor {
         // This set includes our strings defined above as well as some standard flowfile
         // attributes.
         public static final Set<String> IGNORED_ATTRIBUTES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
-                STATUS_CODE, STATUS_MESSAGE, RESPONSE_BODY, REQUEST_URL, TRANSACTION_ID, REMOTE_DN,
-                "uuid", "filename", "path"
-        )));
+            STATUS_CODE, STATUS_MESSAGE, RESPONSE_BODY, REQUEST_URL, TRANSACTION_ID, REMOTE_DN,
+            "uuid", "filename", "path")));
 
         // properties
         public static final PropertyDescriptor PROP_METHOD = new PropertyDescriptor.Builder()
-                .name("HTTP Method")
-                .description("HTTP request method (GET, POST, PUT, DELETE, HEAD, OPTIONS).")
-                .required(true)
-                .defaultValue("GET")
-                .expressionLanguageSupported(true)
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .build();
+            .name("HTTP Method")
+            .description("HTTP request method (GET, POST, PUT, DELETE, HEAD, OPTIONS).")
+            .required(true)
+            .defaultValue("GET")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_URL = new PropertyDescriptor.Builder()
-                .name("Remote URL")
-                .description("Remote URL which will be connected to, including scheme, host, port, path.")
-                .required(true)
-                .expressionLanguageSupported(true)
-                .addValidator(StandardValidators.URL_VALIDATOR)
-                .build();
+            .name("Remote URL")
+            .description("Remote URL which will be connected to, including scheme, host, port, path.")
+            .required(true)
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
-                .name("Connection Timeout")
-                .description("Max wait time for connection to remote service.")
-                .required(true)
-                .defaultValue("5 secs")
-                .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-                .build();
+            .name("Connection Timeout")
+            .description("Max wait time for connection to remote service.")
+            .required(true)
+            .defaultValue("5 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_READ_TIMEOUT = new PropertyDescriptor.Builder()
-                .name("Read Timeout")
-                .description("Max wait time for response from remote service.")
-                .required(true)
-                .defaultValue("15 secs")
-                .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-                .build();
+            .name("Read Timeout")
+            .description("Max wait time for response from remote service.")
+            .required(true)
+            .defaultValue("15 secs")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_DATE_HEADER = new PropertyDescriptor.Builder()
-                .name("Include Date Header")
-                .description("Include an RFC-2616 Date header in the request.")
-                .required(true)
-                .defaultValue("True")
-                .allowableValues("True", "False")
-                .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-                .build();
+            .name("Include Date Header")
+            .description("Include an RFC-2616 Date header in the request.")
+            .required(true)
+            .defaultValue("True")
+            .allowableValues("True", "False")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_FOLLOW_REDIRECTS = new PropertyDescriptor.Builder()
-                .name("Follow Redirects")
-                .description("Follow HTTP redirects issued by remote server.")
-                .required(true)
-                .defaultValue("True")
-                .allowableValues("True", "False")
-                .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-                .build();
+            .name("Follow Redirects")
+            .description("Follow HTTP redirects issued by remote server.")
+            .required(true)
+            .defaultValue("True")
+            .allowableValues("True", "False")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_ATTRIBUTES_TO_SEND = new PropertyDescriptor.Builder()
-                .name("Attributes to Send")
-                .description("Regular expression that defines which attributes to send as HTTP headers in the request. "
-                        + "If not defined, no attributes are sent as headers.")
-                .required(false)
-                .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-                .build();
+            .name("Attributes to Send")
+            .description("Regular expression that defines which attributes to send as HTTP headers in the request. "
+                + "If not defined, no attributes are sent as headers.")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
 
         public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-                .name("SSL Context Service")
-                .description("The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.")
-                .required(false)
-                .identifiesControllerService(SSLContextService.class)
-                .build();
+            .name("SSL Context Service")
+            .description("The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
 
         public static final PropertyDescriptor PROP_PROXY_HOST = new PropertyDescriptor.Builder()
             .name("Proxy Host")
@@ -256,33 +255,33 @@ public final class InvokeHTTP extends AbstractProcessor {
             .build();
 
         // Per RFC 7235, 2617, and 2616.
-        //      basic-credentials   = base64-user-pass
-        //      base64-user-pass    = userid ":" password
-        //      userid              = *<TEXT excluding ":">
-        //      password            = *TEXT
+        // basic-credentials = base64-user-pass
+        // base64-user-pass = userid ":" password
+        // userid = *<TEXT excluding ":">
+        // password = *TEXT
         //
-        //      OCTET          = <any 8-bit sequence of data>
-        //      CTL            = <any US-ASCII control character (octets 0 - 31) and DEL (127)>
-        //      LWS            = [CRLF] 1*( SP | HT )
-        //      TEXT           = <any OCTET except CTLs but including LWS>
+        // OCTET = <any 8-bit sequence of data>
+        // CTL = <any US-ASCII control character (octets 0 - 31) and DEL (127)>
+        // LWS = [CRLF] 1*( SP | HT )
+        // TEXT = <any OCTET except CTLs but including LWS>
         //
         // Per RFC 7230, username & password in URL are now disallowed in HTTP and HTTPS URIs.
         public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
-                .name("Basic Authentication Username")
-                .displayName("Basic Authentication Username")
-                .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
-                .required(false)
-                .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
-                .build();
+            .name("Basic Authentication Username")
+            .displayName("Basic Authentication Username")
+            .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
+            .required(false)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
+            .build();
 
         public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
-                .name("Basic Authentication Password")
-                .displayName("Basic Authentication Password")
-                .description("The password to be used by the client to authenticate against the Remote URL.")
-                .required(false)
-                .sensitive(true)
-                .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
-                .build();
+            .name("Basic Authentication Password")
+            .displayName("Basic Authentication Password")
+            .description("The password to be used by the client to authenticate against the Remote URL.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
+            .build();
 
         public static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
             PROP_METHOD,
@@ -296,48 +295,46 @@ public final class InvokeHTTP extends AbstractProcessor {
             PROP_BASIC_AUTH_USERNAME,
             PROP_BASIC_AUTH_PASSWORD,
             PROP_PROXY_HOST,
-            PROP_PROXY_PORT
-        ));
+            PROP_PROXY_PORT));
 
         // property to allow the hostname verifier to be overridden
         // this is a "hidden" property - it's configured using a dynamic user property
         public static final PropertyDescriptor PROP_TRUSTED_HOSTNAME = new PropertyDescriptor.Builder()
-                .name("Trusted Hostname")
-                .description("Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted "
-                        + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.")
-                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-                .dynamic(true)
-                .build();
+            .name("Trusted Hostname")
+            .description("Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted "
+                + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dynamic(true)
+            .build();
 
         // relationships
         public static final Relationship REL_SUCCESS_REQ = new Relationship.Builder()
-                .name("Original")
-                .description("Original FlowFile will be routed upon success (2xx status codes).")
-                .build();
+            .name("Original")
+            .description("Original FlowFile will be routed upon success (2xx status codes).")
+            .build();
 
         public static final Relationship REL_SUCCESS_RESP = new Relationship.Builder()
-                .name("Response")
-                .description("Response FlowFile will be routed upon success (2xx status codes).")
-                .build();
+            .name("Response")
+            .description("Response FlowFile will be routed upon success (2xx status codes).")
+            .build();
 
         public static final Relationship REL_RETRY = new Relationship.Builder()
-                .name("Retry")
-                .description("FlowFile will be routed on any status code that can be retried (5xx status codes).")
-                .build();
+            .name("Retry")
+            .description("FlowFile will be routed on any status code that can be retried (5xx status codes).")
+            .build();
 
         public static final Relationship REL_NO_RETRY = new Relationship.Builder()
-                .name("No Retry")
-                .description("FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).")
-                .build();
+            .name("No Retry")
+            .description("FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).")
+            .build();
 
         public static final Relationship REL_FAILURE = new Relationship.Builder()
-                .name("Failure")
-                .description("FlowFile will be routed on any type of connection failure, timeout or general exception.")
-                .build();
+            .name("Failure")
+            .description("FlowFile will be routed on any type of connection failure, timeout or general exception.")
+            .build();
 
         public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
-                REL_SUCCESS_REQ, REL_SUCCESS_RESP, REL_RETRY, REL_NO_RETRY, REL_FAILURE
-        )));
+            REL_SUCCESS_REQ, REL_SUCCESS_RESP, REL_RETRY, REL_NO_RETRY, REL_FAILURE)));
 
     }
 
@@ -403,7 +400,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 transfer();
             } catch (final Exception e) {
                 // log exception
-                logger.error("Routing to {} due to exception: {}", new Object[] { REL_FAILURE.getName(), e }, e);
+                logger.error("Routing to {} due to exception: {}", new Object[] {REL_FAILURE.getName(), e}, e);
 
                 // penalize
                 request = session.penalize(request);
@@ -417,7 +414,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                         session.remove(response);
                     }
                 } catch (final Exception e1) {
-                    logger.error("Could not cleanup response flowfile due to exception: {}", new Object[] { e1 }, e1);
+                    logger.error("Could not cleanup response flowfile due to exception: {}", new Object[] {e1}, e1);
                 }
             }
         }
@@ -545,7 +542,7 @@ public final class InvokeHTTP extends AbstractProcessor {
 
                         // emit provenance event
                         final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
-                        session.getProvenanceReporter().modifyContent(response, "Updated content with data received from " + conn.getURL().toExternalForm(), millis);
+                        session.getProvenanceReporter().fetch(response, conn.getURL().toExternalForm(), millis);
                     }
 
                 }
@@ -562,7 +559,7 @@ public final class InvokeHTTP extends AbstractProcessor {
 
             // log the status codes from the response
             logger.info("Request to {} returned status code {} for {}",
-                    new Object[]{conn.getURL().toExternalForm(), statusCode, request});
+                new Object[] {conn.getURL().toExternalForm(), statusCode, request});
 
             // transfer to the correct relationship
             // 2xx -> SUCCESS
@@ -660,12 +657,12 @@ public final class InvokeHTTP extends AbstractProcessor {
 
         private void logRequest() {
             logger.debug("\nRequest to remote service:\n\t{}\n{}",
-                    new Object[]{conn.getURL().toExternalForm(), getLogString(conn.getRequestProperties())});
+                new Object[] {conn.getURL().toExternalForm(), getLogString(conn.getRequestProperties())});
         }
 
         private void logResponse() {
             logger.debug("\nResponse from remote service:\n\t{}\n{}",
-                    new Object[]{conn.getURL().toExternalForm(), getLogString(conn.getHeaderFields())});
+                new Object[] {conn.getURL().toExternalForm(), getLogString(conn.getHeaderFields())});
         }
 
         private String getLogString(Map<String, List<String>> map) {
@@ -753,7 +750,7 @@ public final class InvokeHTTP extends AbstractProcessor {
                 return new BufferedInputStream(is);
 
             } catch (IOException e) {
-                logger.warn("Response stream threw an exception: {}", new Object[]{e}, e);
+                logger.warn("Response stream threw an exception: {}", new Object[] {e}, e);
                 return null;
             }
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/51f56402/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index 46cacca..a4fd3d7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -149,8 +149,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_NO_RETRY, 0);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
 
-        //expected in request status.code and status.message
-        //original flow file (+attributes)??????????
+        // expected in request status.code and status.message
+        // original flow file (+attributes)??????????
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
         bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
         bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
@@ -159,10 +159,10 @@ public class TestInvokeHTTP {
         Assert.assertEquals(expected, actual);
         bundle.assertAttributeEquals("Foo", "Bar");
 
-        //expected in response
-        //status code, status message, all headers from server response --> ff attributes
-        //server response message body into payload of ff
-        //should not contain any original ff attributes
+        // expected in response
+        // status code, status message, all headers from server response --> ff attributes
+        // server response message body into payload of ff
+        // should not contain any original ff attributes
         final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
         bundle1.assertContentEquals("/status/200".getBytes("UTF-8"));
         bundle1.assertAttributeEquals(Config.STATUS_CODE, "200");
@@ -198,8 +198,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_NO_RETRY, 0);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
 
-        //expected in request status.code and status.message
-        //original flow file (+attributes)??????????
+        // expected in request status.code and status.message
+        // original flow file (+attributes)??????????
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_REQ).get(0);
         bundle.assertAttributeEquals(Config.STATUS_CODE, "200");
         bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "OK");
@@ -208,10 +208,10 @@ public class TestInvokeHTTP {
         final String expected = "Hello";
         Assert.assertEquals(expected, actual);
 
-        //expected in response
-        //status code, status message, all headers from server response --> ff attributes
-        //server response message body into payload of ff
-        //should not contain any original ff attributes
+        // expected in response
+        // status code, status message, all headers from server response --> ff attributes
+        // server response message body into payload of ff
+        // should not contain any original ff attributes
         final MockFlowFile bundle1 = runner.getFlowFilesForRelationship(Config.REL_SUCCESS_RESP).get(0);
         final String bundle1Content = new String(bundle1.toByteArray(), StandardCharsets.UTF_8);
         assertTrue(bundle1Content.startsWith(expAuth)); // use startsWith instead of equals so we can ignore line endings
@@ -223,17 +223,17 @@ public class TestInvokeHTTP {
         final List<ProvenanceEventRecord> provEvents = runner.getProvenanceEvents();
         assertEquals(2, provEvents.size());
         boolean forkEvent = false;
-        boolean contentModEvent = false;
+        boolean fetchEvent = false;
         for (final ProvenanceEventRecord event : provEvents) {
             if (event.getEventType() == ProvenanceEventType.FORK) {
                 forkEvent = true;
-            } else if (event.getEventType() == ProvenanceEventType.CONTENT_MODIFIED) {
-                contentModEvent = true;
+            } else if (event.getEventType() == ProvenanceEventType.FETCH) {
+                fetchEvent = true;
             }
         }
 
         assertTrue(forkEvent);
-        assertTrue(contentModEvent);
+        assertTrue(fetchEvent);
     }
 
     @Test
@@ -257,8 +257,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_NO_RETRY, 1);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
 
-        //expected in request status.code and status.message
-        //original flow file (+attributes)??????????
+        // expected in request status.code and status.message
+        // original flow file (+attributes)??????????
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
         bundle.assertAttributeEquals(Config.STATUS_CODE, "401");
         bundle.assertAttributeEquals(Config.STATUS_MESSAGE, "Unauthorized");
@@ -286,7 +286,7 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_NO_RETRY, 0);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
 
-        //expected in response
+        // expected in response
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_RETRY).get(0);
         final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
         bundle.assertAttributeEquals(Config.STATUS_CODE, "500");
@@ -313,8 +313,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_RETRY, 0);
         runner.assertTransferCount(Config.REL_NO_RETRY, 1);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
-        //getMyFlowFiles();
-        //expected in response
+        // getMyFlowFiles();
+        // expected in response
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
         final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
 
@@ -340,8 +340,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_RETRY, 0);
         runner.assertTransferCount(Config.REL_NO_RETRY, 1);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
-        //getMyFlowFiles();
-        //expected in response
+        // getMyFlowFiles();
+        // expected in response
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
         final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
 
@@ -367,8 +367,8 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_RETRY, 0);
         runner.assertTransferCount(Config.REL_NO_RETRY, 1);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
-        //getMyFlowFiles();
-        //expected in response
+        // getMyFlowFiles();
+        // expected in response
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
         final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
 
@@ -397,7 +397,7 @@ public class TestInvokeHTTP {
         runner.assertTransferCount(Config.REL_NO_RETRY, 1);
         runner.assertTransferCount(Config.REL_FAILURE, 0);
 
-        //expected in response
+        // expected in response
         final MockFlowFile bundle = runner.getFlowFilesForRelationship(Config.REL_NO_RETRY).get(0);
         final String actual = new String(bundle.toByteArray(), StandardCharsets.UTF_8);
 
@@ -593,7 +593,7 @@ public class TestInvokeHTTP {
 
         @Override
         public void handle(String target, Request baseRequest,
-                HttpServletRequest request, HttpServletResponse response)
+            HttpServletRequest request, HttpServletResponse response)
                 throws IOException, ServletException {
 
             baseRequest.setHandled(true);


[06/50] [abbrv] nifi git commit: NIFI-673: Initial implementation of ListSFTP, FetchSFTP

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
index f0061b8..fe277df 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.util.List;
 
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.util.StandardValidators;
 
@@ -34,6 +35,8 @@ public interface FileTransfer extends Closeable {
 
     InputStream getInputStream(String remoteFileName) throws IOException;
 
+    InputStream getInputStream(String remoteFileName, FlowFile flowFile) throws IOException;
+
     void flush() throws IOException;
 
     FileInfo getRemoteFileInfo(FlowFile flowFile, String path, String remoteFileName) throws IOException;
@@ -51,127 +54,127 @@ public interface FileTransfer extends Closeable {
     void ensureDirectoryExists(FlowFile flowFile, File remoteDirectory) throws IOException;
 
     public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
-            .name("Hostname")
-            .description("The fully qualified hostname or IP address of the remote system")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(true)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Hostname")
+        .description("The fully qualified hostname or IP address of the remote system")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
-            .name("Username")
-            .description("Username")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(true)
-            .build();
+        .name("Username")
+        .description("Username")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .build();
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
-            .name("Password")
-            .description("Password for the user account")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(false)
-            .sensitive(true)
-            .build();
+        .name("Password")
+        .description("Password for the user account")
+        .addValidator(Validator.VALID)
+        .required(false)
+        .sensitive(true)
+        .build();
     public static final PropertyDescriptor DATA_TIMEOUT = new PropertyDescriptor.Builder()
-            .name("Data Timeout")
-            .description("Amount of time to wait before timing out while transferring data")
-            .required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .defaultValue("30 sec")
-            .build();
+        .name("Data Timeout")
+        .description("When transferring a file between the local and remote system, this value specifies how long is allowed to elapse without any data being transferred between systems")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("30 sec")
+        .build();
     public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
-            .name("Connection Timeout")
-            .description("Amount of time to wait before timing out while creating a connection")
-            .required(true)
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .defaultValue("30 sec")
-            .build();
+        .name("Connection Timeout")
+        .description("Amount of time to wait before timing out while creating a connection")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("30 sec")
+        .build();
     public static final PropertyDescriptor REMOTE_PATH = new PropertyDescriptor.Builder()
-            .name("Remote Path")
-            .description("The path on the remote system from which to pull or push files")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Remote Path")
+        .description("The path on the remote system from which to pull or push files")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor CREATE_DIRECTORY = new PropertyDescriptor.Builder()
-            .name("Create Directory")
-            .description("Specifies whether or not the remote directory should be created if it does not exist.")
-            .required(true)
-            .allowableValues("true", "false")
-            .defaultValue("false")
-            .build();
+        .name("Create Directory")
+        .description("Specifies whether or not the remote directory should be created if it does not exist.")
+        .required(true)
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .build();
 
     public static final PropertyDescriptor USE_COMPRESSION = new PropertyDescriptor.Builder()
-            .name("Use Compression")
-            .description("Indicates whether or not ZLIB compression should be used when transferring files")
-            .allowableValues("true", "false")
-            .defaultValue("false")
-            .required(true)
-            .build();
+        .name("Use Compression")
+        .description("Indicates whether or not ZLIB compression should be used when transferring files")
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .required(true)
+        .build();
 
     // GET-specific properties
     public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
-            .name("Search Recursively")
-            .description("If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories")
-            .required(true)
-            .defaultValue("false")
-            .allowableValues("true", "false")
-            .build();
+        .name("Search Recursively")
+        .description("If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories")
+        .required(true)
+        .defaultValue("false")
+        .allowableValues("true", "false")
+        .build();
     public static final PropertyDescriptor FILE_FILTER_REGEX = new PropertyDescriptor.Builder()
-            .name("File Filter Regex")
-            .description("Provides a Java Regular Expression for filtering Filenames; if a filter is supplied, only files whose names match that Regular Expression will be fetched")
-            .required(false)
-            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .build();
+        .name("File Filter Regex")
+        .description("Provides a Java Regular Expression for filtering Filenames; if a filter is supplied, only files whose names match that Regular Expression will be fetched")
+        .required(false)
+        .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+        .build();
     public static final PropertyDescriptor PATH_FILTER_REGEX = new PropertyDescriptor.Builder()
-            .name("Path Filter Regex")
-            .description("When " + RECURSIVE_SEARCH.getName() + " is true, then only subdirectories whose path matches the given Regular Expression will be scanned")
-            .required(false)
-            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .build();
+        .name("Path Filter Regex")
+        .description("When " + RECURSIVE_SEARCH.getName() + " is true, then only subdirectories whose path matches the given Regular Expression will be scanned")
+        .required(false)
+        .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+        .build();
     public static final PropertyDescriptor MAX_SELECTS = new PropertyDescriptor.Builder()
-            .name("Max Selects")
-            .description("The maximum number of files to pull in a single connection")
-            .defaultValue("100")
-            .required(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .build();
+        .name("Max Selects")
+        .description("The maximum number of files to pull in a single connection")
+        .defaultValue("100")
+        .required(true)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .build();
     public static final PropertyDescriptor REMOTE_POLL_BATCH_SIZE = new PropertyDescriptor.Builder()
-            .name("Remote Poll Batch Size")
-            .description("The value specifies how many file paths to find in a given directory on the remote system when doing a file listing. This value "
-                    + "in general should not need to be modified but when polling against a remote system with a tremendous number of files this value can "
-                    + "be critical.  Setting this value too high can result very poor performance and setting it too low can cause the flow to be slower "
-                    + "than normal.")
-            .defaultValue("5000")
-            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
-            .required(true)
-            .build();
+        .name("Remote Poll Batch Size")
+        .description("The value specifies how many file paths to find in a given directory on the remote system when doing a file listing. This value "
+            + "in general should not need to be modified but when polling against a remote system with a tremendous number of files this value can "
+            + "be critical.  Setting this value too high can result very poor performance and setting it too low can cause the flow to be slower "
+            + "than normal.")
+        .defaultValue("5000")
+        .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+        .required(true)
+        .build();
     public static final PropertyDescriptor DELETE_ORIGINAL = new PropertyDescriptor.Builder()
-            .name("Delete Original")
-            .description("Determines whether or not the file is deleted from the remote system after it has been successfully transferred")
-            .defaultValue("true")
-            .allowableValues("true", "false")
-            .required(true)
-            .build();
+        .name("Delete Original")
+        .description("Determines whether or not the file is deleted from the remote system after it has been successfully transferred")
+        .defaultValue("true")
+        .allowableValues("true", "false")
+        .required(true)
+        .build();
     public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder()
-            .name("Polling Interval")
-            .description("Determines how long to wait between fetching the listing for new files")
-            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-            .required(true)
-            .defaultValue("60 sec")
-            .build();
+        .name("Polling Interval")
+        .description("Determines how long to wait between fetching the listing for new files")
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .required(true)
+        .defaultValue("60 sec")
+        .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")
-            .allowableValues("true", "false")
-            .defaultValue("true")
-            .required(true)
-            .build();
+        .name("Ignore Dotted Files")
+        .description("If true, files whose names begin with a dot (\".\") will be ignored")
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
     public static final PropertyDescriptor USE_NATURAL_ORDERING = new PropertyDescriptor.Builder()
-            .name("Use Natural Ordering")
-            .description("If true, will pull files in the order in which they are naturally listed; otherwise, the order in which the files will be pulled is not defined")
-            .allowableValues("true", "false")
-            .defaultValue("false")
-            .required(true)
-            .build();
+        .name("Use Natural Ordering")
+        .description("If true, will pull files in the order in which they are naturally listed; otherwise, the order in which the files will be pulled is not defined")
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .required(true)
+        .build();
 
     // PUT-specific properties
     public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
@@ -183,77 +186,77 @@ public interface FileTransfer extends Closeable {
     public static final String CONFLICT_RESOLUTION_FAIL = "FAIL";
     public static final String CONFLICT_RESOLUTION_NONE = "NONE";
     public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
-            .name("Conflict Resolution")
-            .description("Determines how to handle the problem of filename collisions")
-            .required(true)
-            .allowableValues(CONFLICT_RESOLUTION_REPLACE, CONFLICT_RESOLUTION_IGNORE, CONFLICT_RESOLUTION_RENAME, CONFLICT_RESOLUTION_REJECT, CONFLICT_RESOLUTION_FAIL, CONFLICT_RESOLUTION_NONE)
-            .defaultValue(CONFLICT_RESOLUTION_NONE)
-            .build();
+        .name("Conflict Resolution")
+        .description("Determines how to handle the problem of filename collisions")
+        .required(true)
+        .allowableValues(CONFLICT_RESOLUTION_REPLACE, CONFLICT_RESOLUTION_IGNORE, CONFLICT_RESOLUTION_RENAME, CONFLICT_RESOLUTION_REJECT, CONFLICT_RESOLUTION_FAIL, CONFLICT_RESOLUTION_NONE)
+        .defaultValue(CONFLICT_RESOLUTION_NONE)
+        .build();
     public static final PropertyDescriptor REJECT_ZERO_BYTE = new PropertyDescriptor.Builder()
-            .name("Reject Zero-Byte Files")
-            .description("Determines whether or not Zero-byte files should be rejected without attempting to transfer")
-            .allowableValues("true", "false")
-            .defaultValue("true")
-            .build();
+        .name("Reject Zero-Byte Files")
+        .description("Determines whether or not Zero-byte files should be rejected without attempting to transfer")
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .build();
     public static final PropertyDescriptor DOT_RENAME = new PropertyDescriptor.Builder()
-            .name("Dot Rename")
-            .description("If true, then the filename of the sent file is prepended with a \".\" and then renamed back to the "
-                    + "original once the file is completely sent. Otherwise, there is no rename. This property is ignored if the "
-                    + "Temporary Filename property is set.")
-            .allowableValues("true", "false")
-            .defaultValue("true")
-            .build();
+        .name("Dot Rename")
+        .description("If true, then the filename of the sent file is prepended with a \".\" and then renamed back to the "
+            + "original once the file is completely sent. Otherwise, there is no rename. This property is ignored if the "
+            + "Temporary Filename property is set.")
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .build();
     public static final PropertyDescriptor TEMP_FILENAME = new PropertyDescriptor.Builder()
-            .name("Temporary Filename")
-            .description("If set, the filename of the sent file will be equal to the value specified during the transfer and after successful "
-                    + "completion will be renamed to the original filename. If this value is set, the Dot Rename property is ignored.")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .required(false)
-            .build();
+        .name("Temporary Filename")
+        .description("If set, the filename of the sent file will be equal to the value specified during the transfer and after successful "
+            + "completion will be renamed to the original filename. If this value is set, the Dot Rename property is ignored.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(false)
+        .build();
     public static final PropertyDescriptor LAST_MODIFIED_TIME = new PropertyDescriptor.Builder()
-            .name("Last Modified Time")
-            .description("The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime will not be changed. "
-                    + "Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as ${file.lastModifiedTime}. If the value "
-                    + "is invalid, the processor will not be invalid but will fail to change lastModifiedTime of the file.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Last Modified Time")
+        .description("The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime will not be changed. "
+            + "Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as ${file.lastModifiedTime}. If the value "
+            + "is invalid, the processor will not be invalid but will fail to change lastModifiedTime of the file.")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor PERMISSIONS = new PropertyDescriptor.Builder()
-            .name("Permissions")
-            .description("The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx with a - in place of "
-                    + "denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the permissions will not be changed. You may "
-                    + "also use expression language such as ${file.permissions}. If the value is invalid, the processor will not be invalid but will "
-                    + "fail to change permissions of the file.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Permissions")
+        .description("The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx with a - in place of "
+            + "denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the permissions will not be changed. You may "
+            + "also use expression language such as ${file.permissions}. If the value is invalid, the processor will not be invalid but will "
+            + "fail to change permissions of the file.")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder()
-            .name("Remote Owner")
-            .description("Integer value representing the User ID to set on the file after transferring it. If not set, the owner will not be set. "
-                    + "You may also use expression language such as ${file.owner}. If the value is invalid, the processor will not be invalid but "
-                    + "will fail to change the owner of the file.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Remote Owner")
+        .description("Integer value representing the User ID to set on the file after transferring it. If not set, the owner will not be set. "
+            + "You may also use expression language such as ${file.owner}. If the value is invalid, the processor will not be invalid but "
+            + "will fail to change the owner of the file.")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
-            .name("Remote Group")
-            .description("Integer value representing the Group ID to set on the file after transferring it. If not set, the group will not be set. "
-                    + "You may also use expression language such as ${file.group}. If the value is invalid, the processor will not be invalid but "
-                    + "will fail to change the group of the file.")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Remote Group")
+        .description("Integer value representing the Group ID to set on the file after transferring it. If not set, the group will not be set. "
+            + "You may also use expression language such as ${file.group}. If the value is invalid, the processor will not be invalid but "
+            + "will fail to change the group of the file.")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
-            .name("Batch Size")
-            .description("The maximum number of FlowFiles to send in a single connection")
-            .required(true)
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .defaultValue("500")
-            .build();
+        .name("Batch Size")
+        .description("The maximum number of FlowFiles to send in a single connection")
+        .required(true)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .defaultValue("500")
+        .build();
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/ListableEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/ListableEntity.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/ListableEntity.java
new file mode 100644
index 0000000..6e019ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/ListableEntity.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard.util;
+
+public interface ListableEntity {
+
+    /**
+     * @return The name of the remote entity
+     */
+    String getName();
+
+    /**
+     * @return the identifier of the remote entity. This may or may not be the same as the name of the
+     *         entity but should be unique across all entities.
+     */
+    String getIdentifier();
+
+
+    /**
+     * @return the timestamp for this entity so that we can be efficient about not performing listings of the same
+     *         entities multiple times
+     */
+    long getTimestamp();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/PermissionDeniedException.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/PermissionDeniedException.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/PermissionDeniedException.java
new file mode 100644
index 0000000..465995e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/PermissionDeniedException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard.util;
+
+import java.io.IOException;
+
+public class PermissionDeniedException extends IOException {
+    private static final long serialVersionUID = -6215434916883053982L;
+
+    public PermissionDeniedException(final String message) {
+        super(message);
+    }
+
+    public PermissionDeniedException(final String message, final Throwable t) {
+        super(message, t);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
index 19955e7..c28f275 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.standard.util;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Path;
@@ -51,45 +52,45 @@ import com.jcraft.jsch.SftpException;
 public class SFTPTransfer implements FileTransfer {
 
     public static final PropertyDescriptor PRIVATE_KEY_PATH = new PropertyDescriptor.Builder()
-            .name("Private Key Path")
-            .description("The fully qualified path to the Private Key file")
-            .required(false)
-            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .build();
+        .name("Private Key Path")
+        .description("The fully qualified path to the Private Key file")
+        .required(false)
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .build();
     public static final PropertyDescriptor PRIVATE_KEY_PASSPHRASE = new PropertyDescriptor.Builder()
-            .name("Private Key Passphrase")
-            .description("Password for the private key")
-            .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .sensitive(true)
-            .build();
+        .name("Private Key Passphrase")
+        .description("Password for the private key")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .sensitive(true)
+        .build();
     public static final PropertyDescriptor HOST_KEY_FILE = new PropertyDescriptor.Builder()
-            .name("Host Key File")
-            .description("If supplied, the given file will be used as the Host Key; otherwise, no use host key file will be used")
-            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-            .required(false)
-            .build();
+        .name("Host Key File")
+        .description("If supplied, the given file will be used as the Host Key; otherwise, no use host key file will be used")
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .required(false)
+        .build();
     public static final PropertyDescriptor STRICT_HOST_KEY_CHECKING = new PropertyDescriptor.Builder()
-            .name("Strict Host Key Checking")
-            .description("Indicates whether or not strict enforcement of hosts keys should be applied")
-            .allowableValues("true", "false")
-            .defaultValue("false")
-            .required(true)
-            .build();
+        .name("Strict Host Key Checking")
+        .description("Indicates whether or not strict enforcement of hosts keys should be applied")
+        .allowableValues("true", "false")
+        .defaultValue("false")
+        .required(true)
+        .build();
     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
-            .name("Port")
-            .description("The port that the remote system is listening on for file transfers")
-            .addValidator(StandardValidators.PORT_VALIDATOR)
-            .required(true)
-            .defaultValue("22")
-            .build();
+        .name("Port")
+        .description("The port that the remote system is listening on for file transfers")
+        .addValidator(StandardValidators.PORT_VALIDATOR)
+        .required(true)
+        .defaultValue("22")
+        .build();
     public static final PropertyDescriptor USE_KEEPALIVE_ON_TIMEOUT = new PropertyDescriptor.Builder()
-            .name("Send Keep Alive On Timeout")
-            .description("Indicates whether or not to send a single Keep Alive message when SSH socket times out")
-            .allowableValues("true", "false")
-            .defaultValue("true")
-            .required(true)
-            .build();
+        .name("Send Keep Alive On Timeout")
+        .description("Indicates whether or not to send a single Keep Alive message when SSH socket times out")
+        .allowableValues("true", "false")
+        .defaultValue("true")
+        .required(true)
+        .build();
 
     /**
      * Dynamic property which is used to decide if the {@link #ensureDirectoryExists(FlowFile, File)} method should perform a {@link ChannelSftp#ls(String)} before calling
@@ -99,12 +100,12 @@ public class SFTPTransfer implements FileTransfer {
      * This property is dynamic until deemed a worthy inclusion as proper.
      */
     public static final PropertyDescriptor DISABLE_DIRECTORY_LISTING = new PropertyDescriptor.Builder()
-            .name("Disable Directory Listing")
-            .description("Disables directory listings before operations which might fail, such as configurations which create directory structures.")
-            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-            .dynamic(true)
-            .defaultValue("false")
-            .build();
+        .name("Disable Directory Listing")
+        .description("Disables directory listings before operations which might fail, such as configurations which create directory structures.")
+        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+        .dynamic(true)
+        .defaultValue("false")
+        .build();
 
     private final ProcessorLog logger;
 
@@ -133,7 +134,16 @@ public class SFTPTransfer implements FileTransfer {
     public List<FileInfo> getListing() throws IOException {
         final String path = ctx.getProperty(FileTransfer.REMOTE_PATH).evaluateAttributeExpressions().getValue();
         final int depth = 0;
-        final int maxResults = ctx.getProperty(FileTransfer.REMOTE_POLL_BATCH_SIZE).asInteger();
+
+        final int maxResults;
+        final PropertyValue batchSizeValue = ctx.getProperty(FileTransfer.REMOTE_POLL_BATCH_SIZE);
+        if (batchSizeValue == null) {
+            maxResults = Integer.MAX_VALUE;
+        } else {
+            final Integer configuredValue = batchSizeValue.asInteger();
+            maxResults = configuredValue == null ? Integer.MAX_VALUE : configuredValue;
+        }
+
         final List<FileInfo> listing = new ArrayList<>(1000);
         getListing(path, depth, maxResults, listing);
         return listing;
@@ -222,7 +232,15 @@ public class SFTPTransfer implements FileTransfer {
                 sftp.ls(path, filter);
             }
         } catch (final SftpException e) {
-            throw new IOException("Failed to obtain file listing for " + (path == null ? "current directory" : path), e);
+            final String pathDesc = path == null ? "current directory" : path;
+            switch (e.id) {
+                case ChannelSftp.SSH_FX_NO_SUCH_FILE:
+                    throw new FileNotFoundException("Could not perform listing on " + pathDesc + " because could not find the file on the remote server");
+                case ChannelSftp.SSH_FX_PERMISSION_DENIED:
+                    throw new PermissionDeniedException("Could not perform listing on " + pathDesc + " due to insufficient permissions");
+                default:
+                    throw new IOException("Failed to obtain file listing for " + pathDesc, e);
+            }
         }
 
         for (final LsEntry entry : subDirs) {
@@ -251,24 +269,36 @@ public class SFTPTransfer implements FileTransfer {
         }
 
         FileInfo.Builder builder = new FileInfo.Builder()
-                .filename(entry.getFilename())
-                .fullPathFileName(newFullForwardPath)
-                .directory(entry.getAttrs().isDir())
-                .size(entry.getAttrs().getSize())
-                .lastModifiedTime(entry.getAttrs().getMTime() * 1000L)
-                .permissions(perms)
-                .owner(Integer.toString(entry.getAttrs().getUId()))
-                .group(Integer.toString(entry.getAttrs().getGId()));
+            .filename(entry.getFilename())
+            .fullPathFileName(newFullForwardPath)
+            .directory(entry.getAttrs().isDir())
+            .size(entry.getAttrs().getSize())
+            .lastModifiedTime(entry.getAttrs().getMTime() * 1000L)
+            .permissions(perms)
+            .owner(Integer.toString(entry.getAttrs().getUId()))
+            .group(Integer.toString(entry.getAttrs().getGId()));
         return builder.build();
     }
 
     @Override
     public InputStream getInputStream(final String remoteFileName) throws IOException {
-        final ChannelSftp sftp = getChannel(null);
+        return getInputStream(remoteFileName, null);
+    }
+
+    @Override
+    public InputStream getInputStream(final String remoteFileName, final FlowFile flowFile) throws IOException {
+        final ChannelSftp sftp = getChannel(flowFile);
         try {
             return sftp.get(remoteFileName);
         } catch (final SftpException e) {
-            throw new IOException("Failed to obtain file content for " + remoteFileName, e);
+            switch (e.id) {
+                case ChannelSftp.SSH_FX_NO_SUCH_FILE:
+                    throw new FileNotFoundException("Could not find file " + remoteFileName + " on remote SFTP Server");
+                case ChannelSftp.SSH_FX_PERMISSION_DENIED:
+                    throw new PermissionDeniedException("Insufficient permissions to read file " + remoteFileName + " from remote SFTP Server", e);
+                default:
+                    throw new IOException("Failed to obtain file content for " + remoteFileName, e);
+            }
         }
     }
 
@@ -283,7 +313,14 @@ public class SFTPTransfer implements FileTransfer {
         try {
             sftp.rm(fullPath);
         } catch (final SftpException e) {
-            throw new IOException("Failed to delete remote file " + fullPath, e);
+            switch (e.id) {
+                case ChannelSftp.SSH_FX_NO_SUCH_FILE:
+                    throw new FileNotFoundException("Could not find file " + remoteFileName + " to remove from remote SFTP Server");
+                case ChannelSftp.SSH_FX_PERMISSION_DENIED:
+                    throw new PermissionDeniedException("Insufficient permissions to delete file " + remoteFileName + " from remote SFTP Server", e);
+                default:
+                    throw new IOException("Failed to delete remote file " + fullPath, e);
+            }
         }
     }
 
@@ -333,10 +370,10 @@ public class SFTPTransfer implements FileTransfer {
             if (directoryName.getParent() != null && !directoryName.getParentFile().equals(new File(File.separator))) {
                 ensureDirectoryExists(flowFile, directoryName.getParentFile());
             }
-            logger.debug("Remote Directory {} does not exist; creating it", new Object[]{remoteDirectory});
+            logger.debug("Remote Directory {} does not exist; creating it", new Object[] {remoteDirectory});
             try {
                 channel.mkdir(remoteDirectory);
-                logger.debug("Created {}", new Object[]{remoteDirectory});
+                logger.debug("Created {}", new Object[] {remoteDirectory});
             } catch (final SftpException e) {
                 throw new IOException("Failed to create remote directory " + remoteDirectory + " due to " + e, e);
             }
@@ -358,9 +395,9 @@ public class SFTPTransfer implements FileTransfer {
 
         final JSch jsch = new JSch();
         try {
-            final Session session = jsch.getSession(ctx.getProperty(USERNAME).getValue(),
-                    ctx.getProperty(HOSTNAME).evaluateAttributeExpressions(flowFile).getValue(),
-                    ctx.getProperty(PORT).evaluateAttributeExpressions(flowFile).asInteger().intValue());
+            final Session session = jsch.getSession(ctx.getProperty(USERNAME).evaluateAttributeExpressions(flowFile).getValue(),
+                ctx.getProperty(HOSTNAME).evaluateAttributeExpressions(flowFile).getValue(),
+                ctx.getProperty(PORT).evaluateAttributeExpressions(flowFile).asInteger().intValue());
 
             final String hostKeyVal = ctx.getProperty(HOST_KEY_FILE).getValue();
             if (hostKeyVal != null) {
@@ -371,7 +408,8 @@ public class SFTPTransfer implements FileTransfer {
             properties.setProperty("StrictHostKeyChecking", ctx.getProperty(STRICT_HOST_KEY_CHECKING).asBoolean() ? "yes" : "no");
             properties.setProperty("PreferredAuthentications", "publickey,password");
 
-            if (ctx.getProperty(FileTransfer.USE_COMPRESSION).asBoolean()) {
+            final PropertyValue compressionValue = ctx.getProperty(FileTransfer.USE_COMPRESSION);
+            if (compressionValue != null && "true".equalsIgnoreCase(compressionValue.getValue())) {
                 properties.setProperty("compression.s2c", "zlib@openssh.com,zlib,none");
                 properties.setProperty("compression.c2s", "zlib@openssh.com,zlib,none");
             } else {
@@ -381,12 +419,12 @@ public class SFTPTransfer implements FileTransfer {
 
             session.setConfig(properties);
 
-            final String privateKeyFile = ctx.getProperty(PRIVATE_KEY_PATH).getValue();
+            final String privateKeyFile = ctx.getProperty(PRIVATE_KEY_PATH).evaluateAttributeExpressions(flowFile).getValue();
             if (privateKeyFile != null) {
-                jsch.addIdentity(privateKeyFile, ctx.getProperty(PRIVATE_KEY_PASSPHRASE).getValue());
+                jsch.addIdentity(privateKeyFile, ctx.getProperty(PRIVATE_KEY_PASSPHRASE).evaluateAttributeExpressions(flowFile).getValue());
             }
 
-            final String password = ctx.getProperty(FileTransfer.PASSWORD).getValue();
+            final String password = ctx.getProperty(FileTransfer.PASSWORD).evaluateAttributeExpressions(flowFile).getValue();
             if (password != null) {
                 session.setPassword(password);
             }
@@ -428,7 +466,7 @@ public class SFTPTransfer implements FileTransfer {
                 sftp.exit();
             }
         } catch (final Exception ex) {
-            logger.warn("Failed to close ChannelSftp due to {}", new Object[]{ex.toString()}, ex);
+            logger.warn("Failed to close ChannelSftp due to {}", new Object[] {ex.toString()}, ex);
         }
         sftp = null;
 
@@ -437,7 +475,7 @@ public class SFTPTransfer implements FileTransfer {
                 session.disconnect();
             }
         } catch (final Exception ex) {
-            logger.warn("Failed to close session due to {}", new Object[]{ex.toString()}, ex);
+            logger.warn("Failed to close session due to {}", new Object[] {ex.toString()}, ex);
         }
         session = null;
     }
@@ -515,7 +553,7 @@ public class SFTPTransfer implements FileTransfer {
                 int time = (int) (fileModifyTime.getTime() / 1000L);
                 sftp.setMtime(tempPath, time);
             } catch (final Exception e) {
-                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[]{tempPath, lastModifiedTime, e});
+                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[] {tempPath, lastModifiedTime, e});
             }
         }
 
@@ -527,7 +565,7 @@ public class SFTPTransfer implements FileTransfer {
                     sftp.chmod(perms, tempPath);
                 }
             } catch (final Exception e) {
-                logger.error("Failed to set permission on {} to {} due to {}", new Object[]{tempPath, permissions, e});
+                logger.error("Failed to set permission on {} to {} due to {}", new Object[] {tempPath, permissions, e});
             }
         }
 
@@ -536,7 +574,7 @@ public class SFTPTransfer implements FileTransfer {
             try {
                 sftp.chown(Integer.parseInt(owner), tempPath);
             } catch (final Exception e) {
-                logger.error("Failed to set owner on {} to {} due to {}", new Object[]{tempPath, owner, e});
+                logger.error("Failed to set owner on {} to {} due to {}", new Object[] {tempPath, owner, e});
             }
         }
 
@@ -545,7 +583,7 @@ public class SFTPTransfer implements FileTransfer {
             try {
                 sftp.chgrp(Integer.parseInt(group), tempPath);
             } catch (final Exception e) {
-                logger.error("Failed to set group on {} to {} due to {}", new Object[]{tempPath, group, e});
+                logger.error("Failed to set group on {} to {} due to {}", new Object[] {tempPath, group, e});
             }
         }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index ff39ad3..b12fb6f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -28,6 +28,7 @@ org.apache.nifi.processors.standard.EvaluateXQuery
 org.apache.nifi.processors.standard.ExecuteStreamCommand
 org.apache.nifi.processors.standard.ExecuteProcess
 org.apache.nifi.processors.standard.ExtractText
+org.apache.nifi.processors.standard.FetchSFTP
 org.apache.nifi.processors.standard.GenerateFlowFile
 org.apache.nifi.processors.standard.GetFile
 org.apache.nifi.processors.standard.GetFTP
@@ -43,6 +44,7 @@ org.apache.nifi.processors.standard.GetJMSQueue
 org.apache.nifi.processors.standard.GetJMSTopic
 org.apache.nifi.processors.standard.ListenHTTP
 org.apache.nifi.processors.standard.ListenUDP
+org.apache.nifi.processors.standard.ListSFTP
 org.apache.nifi.processors.standard.LogAttribute
 org.apache.nifi.processors.standard.MergeContent
 org.apache.nifi.processors.standard.ModifyBytes

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java
new file mode 100644
index 0000000..ba84939
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAbstractListProcessor.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyDescriptor;
+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.processor.ProcessContext;
+import org.apache.nifi.processors.standard.util.ListableEntity;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+
+public class TestAbstractListProcessor {
+
+    @Test
+    public void testOnlyNewEntriesEmitted() {
+        final ConcreteListProcessor proc = new ConcreteListProcessor();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        proc.addEntity("name", "id", 1492L);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 1);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1492L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 1);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1492L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id3", 1491L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1492L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1493L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 1);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1493L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id2", 1493L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 0);
+        runner.clearTransferState();
+
+        proc.addEntity("name", "id", 1494L);
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ConcreteListProcessor.REL_SUCCESS, 1);
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testStateStoredInDistributedService() throws InitializationException {
+        final ConcreteListProcessor proc = new ConcreteListProcessor();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        final DistributedCache cache = new DistributedCache();
+        runner.addControllerService("cache", cache);
+        runner.enableControllerService(cache);
+        runner.setProperty(AbstractListProcessor.DISTRIBUTED_CACHE_SERVICE, "cache");
+
+        runner.run();
+
+        proc.addEntity("name", "id", 1492L);
+        runner.run();
+
+        assertEquals(1, cache.stored.size());
+    }
+
+    @Test
+    public void testFetchOnStart() throws InitializationException {
+        final ConcreteListProcessor proc = new ConcreteListProcessor();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        final DistributedCache cache = new DistributedCache();
+        runner.addControllerService("cache", cache);
+        runner.enableControllerService(cache);
+        runner.setProperty(AbstractListProcessor.DISTRIBUTED_CACHE_SERVICE, "cache");
+
+        runner.run();
+
+        assertEquals(1, cache.fetchCount);
+    }
+
+    private static class DistributedCache extends AbstractControllerService implements DistributedMapCacheClient {
+        private final Map<Object, Object> stored = new HashMap<>();
+        private int fetchCount = 0;
+
+        @Override
+        public <K, V> boolean putIfAbsent(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+            return false;
+        }
+
+        @Override
+        public <K, V> V getAndPutIfAbsent(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer, Deserializer<V> valueDeserializer) throws IOException {
+            return null;
+        }
+
+        @Override
+        public <K> boolean containsKey(K key, Serializer<K> keySerializer) throws IOException {
+            return false;
+        }
+
+        @Override
+        public <K, V> void put(K key, V value, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+            stored.put(key, value);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V get(K key, Serializer<K> keySerializer, Deserializer<V> valueDeserializer) throws IOException {
+            fetchCount++;
+            return (V) stored.get(key);
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public <K> boolean remove(K key, Serializer<K> serializer) throws IOException {
+            final Object value = stored.remove(key);
+            return value != null;
+        }
+    }
+
+
+    private static class ConcreteListProcessor extends AbstractListProcessor<ListableEntity> {
+        private final List<ListableEntity> entities = new ArrayList<>();
+
+        @Override
+        protected File getPersistenceFile() {
+            return new File("target/ListProcessor-local-state.json");
+        }
+
+        public void addEntity(final String name, final String identifier, final long timestamp) {
+            final ListableEntity entity = new ListableEntity() {
+                @Override
+                public String getName() {
+                    return name;
+                }
+
+                @Override
+                public String getIdentifier() {
+                    return identifier;
+                }
+
+                @Override
+                public long getTimestamp() {
+                    return timestamp;
+                }
+            };
+
+            entities.add(entity);
+        }
+
+        @Override
+        protected Map<String, String> createAttributes(final ListableEntity entity, final ProcessContext context) {
+            return Collections.emptyMap();
+        }
+
+        @Override
+        protected String getPath(final ProcessContext context) {
+            return "/path";
+        }
+
+        @Override
+        protected List<ListableEntity> performListing(final ProcessContext context, final Long minTimestamp) throws IOException {
+            return Collections.unmodifiableList(entities);
+        }
+
+        @Override
+        protected boolean isListingResetNecessary(PropertyDescriptor property) {
+            return false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
new file mode 100644
index 0000000..7aa8f9c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import static org.junit.Assert.assertFalse;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.standard.util.FileInfo;
+import org.apache.nifi.processors.standard.util.FileTransfer;
+import org.apache.nifi.processors.standard.util.PermissionDeniedException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+
+public class TestFetchFileTransfer {
+
+    @Test
+    public void testContentFetched() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+        assertFalse(proc.closed);
+        runner.getFlowFilesForRelationship(FetchFileTransfer.REL_SUCCESS).get(0).assertContentEquals("world");
+    }
+
+    @Test
+    public void testContentNotFound() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_NOT_FOUND, 1);
+    }
+
+    @Test
+    public void testInsufficientPermissions() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+
+        proc.addContent("hello.txt", "world".getBytes());
+        proc.allowAccess = false;
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_PERMISSION_DENIED, 1);
+    }
+
+    private static class TestableFetchFileTransfer extends FetchFileTransfer {
+        private boolean allowAccess = true;
+        private boolean closed = false;
+        private final Map<String, byte[]> fileContents = new HashMap<>();
+
+        public void addContent(final String filename, final byte[] content) {
+            this.fileContents.put(filename, content);
+        }
+
+        @Override
+        protected FileTransfer createFileTransfer(final ProcessContext context) {
+            return new FileTransfer() {
+                @Override
+                public void close() throws IOException {
+                    closed = true;
+                }
+
+                @Override
+                public String getHomeDirectory(FlowFile flowFile) throws IOException {
+                    return null;
+                }
+
+                @Override
+                public List<FileInfo> getListing() throws IOException {
+                    return null;
+                }
+
+                @Override
+                public InputStream getInputStream(final String remoteFileName) throws IOException {
+                    return getInputStream(remoteFileName, null);
+                }
+
+                @Override
+                public InputStream getInputStream(String remoteFileName, FlowFile flowFile) throws IOException {
+                    if (!allowAccess) {
+                        throw new PermissionDeniedException("test permission denied");
+                    }
+
+                    final byte[] content = fileContents.get(remoteFileName);
+                    if (content == null) {
+                        throw new FileNotFoundException();
+                    }
+
+                    return new ByteArrayInputStream(content);
+                }
+
+                @Override
+                public void flush() throws IOException {
+                }
+
+                @Override
+                public FileInfo getRemoteFileInfo(FlowFile flowFile, String path, String remoteFileName) throws IOException {
+                    return null;
+                }
+
+                @Override
+                public String put(FlowFile flowFile, String path, String filename, InputStream content) throws IOException {
+                    return null;
+                }
+
+                @Override
+                public void deleteFile(String path, String remoteFileName) throws IOException {
+                    if (!fileContents.containsKey(remoteFileName)) {
+                        throw new FileNotFoundException();
+                    }
+
+                    fileContents.remove(remoteFileName);
+                }
+
+                @Override
+                public void deleteDirectory(String remoteDirectoryName) throws IOException {
+
+                }
+
+                @Override
+                public boolean isClosed() {
+                    return false;
+                }
+
+                @Override
+                public String getProtocolName() {
+                    return "test";
+                }
+
+                @Override
+                public void ensureDirectoryExists(FlowFile flowFile, File remoteDirectory) throws IOException {
+
+                }
+            };
+        }
+    }
+}


[07/50] [abbrv] nifi git commit: NIFI-673: Initial implementation of ListSFTP, FetchSFTP

Posted by ma...@apache.org.
NIFI-673: Initial implementation of ListSFTP, FetchSFTP


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

Branch: refs/heads/NIFI-730
Commit: d1d57931bf996a230ab7941cb6c1524286c97606
Parents: 8a80060
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 4 15:48:28 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:13:02 2015 -0400

----------------------------------------------------------------------
 .../standard/AbstractListProcessor.java         | 505 +++++++++++++++++++
 .../processors/standard/FetchFileTransfer.java  | 296 +++++++++++
 .../nifi/processors/standard/FetchSFTP.java     |  89 ++++
 .../processors/standard/ListFileTransfer.java   | 103 ++++
 .../nifi/processors/standard/ListSFTP.java      |  81 +++
 .../processors/standard/util/EntityListing.java |  71 +++
 .../processors/standard/util/FTPTransfer.java   | 135 ++---
 .../nifi/processors/standard/util/FileInfo.java |  18 +-
 .../processors/standard/util/FileTransfer.java  | 335 ++++++------
 .../standard/util/ListableEntity.java           |  40 ++
 .../util/PermissionDeniedException.java         |  32 ++
 .../processors/standard/util/SFTPTransfer.java  | 174 ++++---
 .../org.apache.nifi.processor.Processor         |   2 +
 .../standard/TestAbstractListProcessor.java     | 221 ++++++++
 .../standard/TestFetchFileTransfer.java         | 186 +++++++
 15 files changed, 1988 insertions(+), 300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
new file mode 100644
index 0000000..8a7fade
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+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;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.standard.util.EntityListing;
+import org.apache.nifi.processors.standard.util.ListableEntity;
+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;
+
+/**
+ * <p>
+ * An Abstract Processor that is intended to simplify the coding required in order to perform Listing operations of remote resources.
+ * Those remote resources may be files, "objects", "messages", or any other sort of entity that may need to be listed in such a way that
+ * we identity the entity only once. Each of these objects, messages, etc. is referred to as an "entity" for the scope of this Processor.
+ * </p>
+ *
+ * <p>
+ * This class is responsible for triggering the listing to occur, filtering the results returned such that only new (unlisted) entities
+ * or entities that have been modified will be emitted from the Processor.
+ * </p>
+ *
+ * <p>
+ * In order to make use of this abstract class, the entities listed must meet the following criteria:
+ * <ul>
+ * <li>
+ * Entity must have a timestamp associated with it. This timestamp is used to determine if entities are "new" or not. Any entity that is
+ * returned by the listing will be considered "new" if the timestamp is later than the latest timestamp pulled.
+ * </li>
+ * <li>
+ * Entity must have a unique identifier. This is used in conjunction with the timestamp in order to determine whether or not the entity is
+ * new. If the timestamp of an entity is before the latest timestamp pulled, then the entity is not considered new. If the timestamp is later
+ * than the last timestamp pulled, then the entity is considered new. If the timestamp is equal to the latest timestamp pulled, then the entity's
+ * identifier is compared to all of the entity identifiers that have that same timestamp in order to determine whether or not the entity has been
+ * seen already.
+ * </li>
+ * <li>
+ * Entity must have a user-readable name that can be used for logging purposes.
+ * </li>
+ * </p>
+ *
+ * <p>
+ * This class persists state across restarts so that even if NiFi is restarted, duplicates will not be pulled from the remote system. This is performed using
+ * two different mechanisms. First, state is stored locally. This allows the system to be restarted and begin processing where it left off. The state that is
+ * stored is the latest timestamp that has been pulled (as determined by the timestamps of the entities that are returned), as well as the unique identifier of
+ * each entity that has that timestamp. See the section above for information about how these pieces of information are used in order to determine entity uniqueness.
+ * </p>
+ *
+ * <p>
+ * In addition to storing state locally, the Processor exposes an optional <code>Distributed Cache Service</code> property. In standalone deployment of NiFi, this is
+ * not necessary. However, in a clustered environment, subclasses of this class are expected to be run only on primary node. While this means that the local state is
+ * accurate as long as the primary node remains constant, the primary node in the cluster can be changed. As a result, if running in a clustered environment, it is
+ * recommended that this property be set. This allows the same state that is described above to also be replicated across the cluster. If this property is set, then
+ * on restart the Processor will not begin listing until it has retrieved an updated state from this service, as it does not know whether or not another node has
+ * modified the state in the mean time.
+ * </p>
+ *
+ * <p>
+ * For each new entity that is listed, the Processor will send a FlowFile to the 'success' relationship. The FlowFile will have no content but will have some set
+ * of attributes (defined by the concrete implementation) that can be used to fetch those remote resources or interact with them in whatever way makes sense for
+ * the configured dataflow.
+ * </p>
+ *
+ * <p>
+ * Subclasses are responsible for the following:
+ *
+ * <ul>
+ * <li>
+ * Perform a listing of remote resources. The subclass will implement the {@link #performListing(ProcessContext, Long)} method, which creates a listing of all
+ * entities on the remote system that have timestamps later than the provided timestamp. If the entities returned have a timestamp before the provided one, those
+ * entities will be filtered out. It is therefore not necessary to perform the filtering of timestamps but is provided in order to give the implementation the ability
+ * to filter those resources on the server side rather than pulling back all of the information, if it makes sense to do so in the concrete implementation.
+ * </li>
+ * <li>
+ * Creating a Map of attributes that are applicable for an entity. The attributes that are assigned to each FlowFile are exactly those returned by the
+ * {@link #createAttributes(ListableEntity, ProcessContext)}.
+ * </li>
+ * <li>
+ * Returning the configured path. Many resources can be comprised of a "path" (or a "container" or "bucket", etc.) as well as name or identifier that is unique only
+ * within that path. The {@link #getPath(ProcessContext)} method is responsible for returning the path that is currently being polled for entities. If this does concept
+ * does not apply for the concrete implementation, it is recommended that the concrete implementation return "." or "/" for all invocations of this method.
+ * </li>
+ * <li>
+ * Determining when the listing must be cleared. It is sometimes necessary to clear state about which entities have already been ingested, as the result of a user
+ * changing a property value. The {@link #isListingResetNecessary(PropertyDescriptor)} method is responsible for determining when the listing needs to be reset by returning
+ * a boolean indicating whether or not a change in the value of the provided property should trigger the timestamp and identifier information to be cleared.
+ * </li>
+ * </ul>
+ * </p>
+ */
+@TriggerSerially
+public abstract class AbstractListProcessor<T extends ListableEntity> extends AbstractProcessor {
+    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 the remote server so that if a new node "
+            + "begins pulling data, it won't duplicate all of the work that has been done. If not specified, the information will not be shared across the cluster. "
+            + "This property does not need to be set for standalone instances of NiFi but should be configured if NiFi is run within a cluster.")
+        .required(false)
+        .identifiesControllerService(DistributedMapCacheClient.class)
+        .build();
+
+
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .description("All FlowFiles that are received are routed to success")
+        .build();
+
+
+    private volatile Long lastListingTime = null;
+    private volatile Set<String> latestIdentifiersListed = new HashSet<>();
+    private volatile boolean electedPrimaryNode = false;
+
+    protected File getPersistenceFile() {
+        return new File("conf/state/" + getIdentifier());
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DISTRIBUTED_CACHE_SERVICE);
+        return properties;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (isListingResetNecessary(descriptor)) {
+            lastListingTime = null; // clear lastListingTime so that we have to fetch new time
+            latestIdentifiersListed = new HashSet<>();
+        }
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        return relationships;
+    }
+
+    protected String getKey(final String directory) {
+        return getIdentifier() + ".lastListingTime." + directory;
+    }
+
+    @OnPrimaryNodeStateChange
+    public void onPrimaryNodeChange(final PrimaryNodeState newState) {
+        if (newState == PrimaryNodeState.ELECTED_PRIMARY_NODE) {
+            electedPrimaryNode = true;
+        }
+    }
+
+    private EntityListing deserialize(final String serializedState) throws JsonParseException, JsonMappingException, IOException {
+        final ObjectMapper mapper = new ObjectMapper();
+        final JsonNode jsonNode = mapper.readTree(serializedState);
+        return mapper.readValue(jsonNode, EntityListing.class);
+    }
+
+
+    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.
+            if (client != null) {
+                try {
+                    final StringSerDe serde = new StringSerDe();
+                    final String serializedState = client.get(getKey(directory), serde, serde);
+                    if (serializedState == null || serializedState.isEmpty()) {
+                        minTimestamp = null;
+                        this.latestIdentifiersListed = Collections.emptySet();
+                    } else {
+                        final EntityListing listing = deserialize(serializedState);
+                        this.lastListingTime = listing.getLatestTimestamp().getTime();
+                        minTimestamp = listing.getLatestTimestamp().getTime();
+                        this.latestIdentifiersListed = new HashSet<>(listing.getMatchingIdentifiers());
+                    }
+
+                    this.lastListingTime = minTimestamp;
+                    electedPrimaryNode = false; // no requirement to pull an update from the distributed cache anymore.
+                } catch (final IOException ioe) {
+                    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();
+                        props.load(fis);
+
+                        // get the local timestamp for this directory, if it exists.
+                        final String locallyPersistedValue = props.getProperty(directory);
+                        if (locallyPersistedValue != null) {
+                            final EntityListing 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 (client != null && (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 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);
+            }
+        }
+
+        return minTimestamp;
+    }
+
+
+    private String serializeState(final List<T> entities) 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 (entities.isEmpty()) {
+            return null;
+        } else {
+            final List<T> sortedEntities = new ArrayList<>(entities);
+            Collections.sort(sortedEntities, new Comparator<ListableEntity>() {
+                @Override
+                public int compare(final ListableEntity o1, final ListableEntity o2) {
+                    return Long.compare(o1.getTimestamp(), o2.getTimestamp());
+                }
+            });
+
+            final long latestListingModTime = sortedEntities.get(sortedEntities.size() - 1).getTimestamp();
+            final Set<String> idsWithTimestampEqualToListingTime = new HashSet<>();
+            for (int i = sortedEntities.size() - 1; i >= 0; i--) {
+                final ListableEntity entity = sortedEntities.get(i);
+                if (entity.getTimestamp() == latestListingModTime) {
+                    idsWithTimestampEqualToListingTime.add(entity.getIdentifier());
+                }
+            }
+
+            this.latestIdentifiersListed = idsWithTimestampEqualToListingTime;
+
+            final EntityListing listing = new EntityListing();
+            listing.setLatestTimestamp(new Date(latestListingModTime));
+            final Set<String> ids = new HashSet<>();
+            for (final String id : idsWithTimestampEqualToListingTime) {
+                ids.add(id);
+            }
+            listing.setMatchingIdentifiers(ids);
+
+            final ObjectMapper mapper = new ObjectMapper();
+            final String serializedState = mapper.writerWithType(EntityListing.class).writeValueAsString(listing);
+            return serializedState;
+        }
+    }
+
+    protected void persistLocalState(final String path, 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");
+        }
+
+        final Properties props = new Properties();
+        if (persistenceFile.exists()) {
+            try (final FileInputStream fis = new FileInputStream(persistenceFile)) {
+                props.load(fis);
+            }
+        }
+
+        props.setProperty(path, serializedState);
+
+        try (final FileOutputStream fos = new FileOutputStream(persistenceFile)) {
+            props.store(fos, null);
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final String path = getPath(context);
+        final DistributedMapCacheClient client = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
+
+        final Long minTimestamp;
+        try {
+            minTimestamp = getMinTimestamp(path, 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;
+        }
+
+        final List<T> entityList;
+        try {
+            entityList = performListing(context, minTimestamp);
+        } catch (final IOException e) {
+            getLogger().error("Failed to perform listing on remote host due to {}", e);
+            context.yield();
+            return;
+        }
+
+        if (entityList == null) {
+            context.yield();
+            return;
+        }
+
+        int listCount = 0;
+        Long latestListingTimestamp = null;
+        for (final T entity : entityList) {
+            final boolean list = (minTimestamp == null || entity.getTimestamp() > minTimestamp ||
+                (entity.getTimestamp() == minTimestamp && !latestIdentifiersListed.contains(entity.getIdentifier())));
+
+            // Create the FlowFile for this path.
+            if (list) {
+                final Map<String, String> attributes = createAttributes(entity, context);
+                FlowFile flowFile = session.create();
+                flowFile = session.putAllAttributes(flowFile, attributes);
+                session.transfer(flowFile, REL_SUCCESS);
+                listCount++;
+
+                if (latestListingTimestamp == null || entity.getTimestamp() > latestListingTimestamp) {
+                    latestListingTimestamp = entity.getTimestamp();
+                }
+            }
+        }
+
+        if (listCount > 0) {
+            getLogger().info("Successfully created listing with {} new objects", 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(entityList);
+            } catch (final Exception e) {
+                getLogger().error("Failed to serialize state due to {}", new Object[] {e});
+            }
+
+            if (serializedState != null) {
+                // Save our state locally.
+                try {
+                    persistLocalState(path, 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.
+                if (client != null) {
+                    try {
+                        client.put(getKey(path), 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 = latestListingTimestamp;
+        } 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;
+        }
+    }
+
+
+    /**
+     * Creates a Map of attributes that should be applied to the FlowFile to represent this entity. This processor will emit a FlowFile for each "new" entity
+     * (see the documentation for this class for a discussion of how this class determines whether or not an entity is "new"). The FlowFile will contain no
+     * content. The attributes that will be included are exactly the attributes that are returned by this method.
+     *
+     * @param entity the entity represented by the FlowFile
+     * @param context the ProcessContext for obtaining configuration information
+     * @return a Map of attributes for this entity
+     */
+    protected abstract Map<String, String> createAttributes(T entity, ProcessContext context);
+
+    /**
+     * Returns the path to perform a listing on.
+     * Many resources can be comprised of a "path" (or a "container" or "bucket", etc.) as well as name or identifier that is unique only
+     * within that path. This method is responsible for returning the path that is currently being polled for entities. If this does concept
+     * does not apply for the concrete implementation, it is recommended that the concrete implementation return "." or "/" for all invocations of this method.
+     *
+     * @param context the ProcessContex to use in order to obtain configuration
+     * @return the path that is to be used to perform the listing, or <code>null</code> if not applicable.
+     */
+    protected abstract String getPath(final ProcessContext context);
+
+    /**
+     * Performs a listing of the remote entities that can be pulled. If any entity that is returned has already been "discovered" or "emitted"
+     * by this Processor, it will be ignored. A discussion of how the Processor determines those entities that have already been emitted is
+     * provided above in the documentation for this class. Any entity that is returned by this method with a timestamp prior to the minTimestamp
+     * will be filtered out by the Processor. Therefore, it is not necessary that implementations perform this filtering but can be more efficient
+     * if the filtering can be performed on the server side prior to retrieving the information.
+     *
+     * @param context the ProcessContex to use in order to pull the appropriate entities
+     * @param minTimestamp the minimum timestamp of entities that should be returned.
+     *
+     * @return a Listing of entities that have a timestamp >= minTimestamp
+     */
+    protected abstract List<T> performListing(final ProcessContext context, final Long minTimestamp) throws IOException;
+
+    /**
+     * Determines whether or not the listing must be reset if the value of the given property is changed
+     *
+     * @param property the property that has changed
+     * @return <code>true</code> if a change in value of the given property necessitates that the listing be reset, <code>false</code> otherwise.
+     */
+    protected abstract boolean isListingResetNecessary(final PropertyDescriptor property);
+
+
+
+    private static 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/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
new file mode 100644
index 0000000..5eecac3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.util.FileTransfer;
+import org.apache.nifi.processors.standard.util.PermissionDeniedException;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.Tuple;
+
+/**
+ * A base class for FetchSFTP, FetchFTP processors
+ */
+public abstract class FetchFileTransfer extends AbstractProcessor {
+    static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
+        .name("Hostname")
+        .description("The fully-qualified hostname or IP address of the host to fetch the data from")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
+    static final PropertyDescriptor UNDEFAULTED_PORT = new PropertyDescriptor.Builder()
+        .name("Port")
+        .description("The port to connect to on the remote host to fetch the data from")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
+    public static final PropertyDescriptor REMOTE_FILENAME = new PropertyDescriptor.Builder()
+        .name("Remote File")
+        .description("The fully qualified filename on the remote system")
+        .required(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .build();
+    public static final PropertyDescriptor DELETE_ORIGINAL = new PropertyDescriptor.Builder()
+        .name("Delete Original")
+        .description("Determines whether or not the file is deleted from the remote system after it has been successfully transferred")
+        .defaultValue("true")
+        .allowableValues("true", "false")
+        .required(true)
+        .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .description("All FlowFiles that are received are routed to success")
+        .build();
+    static final Relationship REL_COMMS_FAILURE = new Relationship.Builder()
+        .name("comms.failure")
+        .description("Any FlowFile that could not be fetched from the remote server due to a communications failure will be transferred to this Relationship.")
+        .build();
+    static final Relationship REL_NOT_FOUND = new Relationship.Builder()
+        .name("not.found")
+        .description("Any FlowFile for which we receive a 'Not Found' message from the remote server will be transferred to this Relationship.")
+        .build();
+    static final Relationship REL_PERMISSION_DENIED = new Relationship.Builder()
+        .name("permission.denied")
+        .description("Any FlowFile that could not be fetched from the remote server due to insufficient permissions will be transferred to this Relationship.")
+        .build();
+
+    private final Map<Tuple<String, Integer>, BlockingQueue<FileTransferIdleWrapper>> fileTransferMap = new HashMap<>();
+    private final long IDLE_CONNECTION_MILLIS = TimeUnit.SECONDS.toMillis(10L); // amount of time to wait before closing an idle connection
+    private volatile long lastClearTime = System.currentTimeMillis();
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_NOT_FOUND);
+        relationships.add(REL_PERMISSION_DENIED);
+        relationships.add(REL_COMMS_FAILURE);
+        return relationships;
+    }
+
+    /**
+     * Close connections that are idle or optionally close all connections.
+     * Connections are considered "idle" if they have not been used in 10 seconds.
+     *
+     * @param closeNonIdleConnections if <code>true</code> will close all connection; if <code>false</code> will close only idle connections
+     */
+    private void closeConnections(final boolean closeNonIdleConnections) {
+        for (final Map.Entry<Tuple<String, Integer>, BlockingQueue<FileTransferIdleWrapper>> entry : fileTransferMap.entrySet()) {
+            final BlockingQueue<FileTransferIdleWrapper> wrapperQueue = entry.getValue();
+
+            final List<FileTransferIdleWrapper> putBack = new ArrayList<>();
+            FileTransferIdleWrapper wrapper;
+            while ((wrapper = wrapperQueue.poll()) != null) {
+                final long lastUsed = wrapper.getLastUsed();
+                final long nanosSinceLastUse = System.nanoTime() - lastUsed;
+                if (!closeNonIdleConnections && TimeUnit.NANOSECONDS.toMillis(nanosSinceLastUse) < IDLE_CONNECTION_MILLIS) {
+                    putBack.add(wrapper);
+                } else {
+                    try {
+                        wrapper.getFileTransfer().close();
+                    } catch (final IOException ioe) {
+                        getLogger().warn("Failed to close Idle Connection due to {}", new Object[] {ioe}, ioe);
+                    }
+                }
+            }
+
+            for (final FileTransferIdleWrapper toPutBack : putBack) {
+                wrapperQueue.offer(toPutBack);
+            }
+        }
+    }
+
+    @OnStopped
+    public void cleanup() {
+        // close all connections
+        closeConnections(true);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(HOSTNAME);
+        properties.add(UNDEFAULTED_PORT);
+        properties.add(REMOTE_FILENAME);
+        properties.add(DELETE_ORIGINAL);
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final StopWatch stopWatch = new StopWatch(true);
+        final String host = context.getProperty(HOSTNAME).evaluateAttributeExpressions(flowFile).getValue();
+        final int port = context.getProperty(UNDEFAULTED_PORT).evaluateAttributeExpressions(flowFile).asInteger();
+        final String filename = context.getProperty(REMOTE_FILENAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        // Try to get a FileTransfer object from our cache.
+        BlockingQueue<FileTransferIdleWrapper> transferQueue;
+        synchronized (fileTransferMap) {
+            final Tuple<String, Integer> tuple = new Tuple<>(host, port);
+
+            transferQueue = fileTransferMap.get(tuple);
+            if (transferQueue == null) {
+                transferQueue = new LinkedBlockingQueue<>();
+                fileTransferMap.put(tuple, transferQueue);
+            }
+
+            // periodically close idle connections
+            if (System.currentTimeMillis() - lastClearTime > IDLE_CONNECTION_MILLIS) {
+                closeConnections(false);
+                lastClearTime = System.currentTimeMillis();
+            }
+        }
+
+        // we have a queue of FileTransfer Objects. Get one from the queue or create a new one.
+        FileTransfer transfer;
+        FileTransferIdleWrapper transferWrapper = transferQueue.poll();
+        if (transferWrapper == null) {
+            transfer = createFileTransfer(context);
+        } else {
+            transfer = transferWrapper.getFileTransfer();
+        }
+
+        // Pull data from remote system.
+        final InputStream in;
+        try {
+            in = transfer.getInputStream(filename, flowFile);
+            flowFile = session.write(flowFile, new OutputStreamCallback() {
+                @Override
+                public void process(final OutputStream out) throws IOException {
+                    StreamUtils.copy(in, out);
+                }
+            });
+            transfer.flush();
+            transferQueue.offer(new FileTransferIdleWrapper(transfer, System.nanoTime()));
+        } catch (final FileNotFoundException e) {
+            getLogger().error("Failed to fetch content for {} from filename {} on remote host {} because the file could not be found on the remote system; routing to {}",
+                new Object[] {flowFile, filename, host, REL_NOT_FOUND.getName()});
+            session.transfer(session.penalize(flowFile), REL_NOT_FOUND);
+            session.getProvenanceReporter().route(flowFile, REL_NOT_FOUND);
+            return;
+        } catch (final PermissionDeniedException e) {
+            getLogger().error("Failed to fetch content for {} from filename {} on remote host {} due to insufficient permissions; routing to {}",
+                new Object[] {flowFile, filename, host, REL_PERMISSION_DENIED.getName()});
+            session.transfer(session.penalize(flowFile), REL_PERMISSION_DENIED);
+            session.getProvenanceReporter().route(flowFile, REL_PERMISSION_DENIED);
+            return;
+        } catch (final IOException e) {
+            try {
+                transfer.close();
+            } catch (final IOException e1) {
+                getLogger().warn("Failed to close connection to {}:{} due to {}", new Object[] {host, port, e.toString()}, e);
+            }
+
+            getLogger().error("Failed to fetch content for {} from filename {} on remote host {}:{} due to {}; routing to failure",
+                new Object[] {flowFile, filename, host, port, e.toString()}, e);
+            session.transfer(session.penalize(flowFile), REL_COMMS_FAILURE);
+            return;
+        }
+
+        // Add FlowFile attributes
+        final String protocolName = transfer.getProtocolName();
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(protocolName + ".remote.host", host);
+        attributes.put(protocolName + ".remote.port", String.valueOf(port));
+        attributes.put(protocolName + ".remote.filename", filename);
+        attributes.put(CoreAttributes.FILENAME.key(), filename);
+        flowFile = session.putAllAttributes(flowFile, attributes);
+
+        // emit provenance event and transfer FlowFile
+        session.getProvenanceReporter().modifyContent(flowFile, "Content replaced with content from " + protocolName + "://" + host + ":" + port + "/" + filename,
+            stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+        session.transfer(flowFile, REL_SUCCESS);
+
+        // delete remote file is necessary
+        final boolean deleteOriginal = context.getProperty(DELETE_ORIGINAL).asBoolean();
+        if (deleteOriginal) {
+            try {
+                transfer.deleteFile(null, filename);
+            } catch (final FileNotFoundException e) {
+                // file doesn't exist -- effectively the same as removing it. Move on.
+            } catch (final IOException ioe) {
+                getLogger().warn("Successfully fetched the content for {} from {}:{}{} but failed to remove the remote file due to {}", new Object[] {flowFile, host, port, filename, ioe}, ioe);
+            }
+        }
+    }
+
+
+    /**
+     * Creates a new instance of a FileTransfer that can be used to pull files from a remote system.
+     *
+     * @param context the ProcessContext to use in order to obtain configured properties
+     * @return a FileTransfer that can be used to pull files from a remote system
+     */
+    protected abstract FileTransfer createFileTransfer(ProcessContext context);
+
+    /**
+     * Wrapper around a FileTransfer object that is used to know when the FileTransfer was last used, so that
+     * we have the ability to close connections that are "idle," or unused for some period of time.
+     */
+    private static class FileTransferIdleWrapper {
+        private final FileTransfer fileTransfer;
+        private final long lastUsed;
+
+        public FileTransferIdleWrapper(final FileTransfer fileTransfer, final long lastUsed) {
+            this.fileTransfer = fileTransfer;
+            this.lastUsed = lastUsed;
+        }
+
+        public FileTransfer getFileTransfer() {
+            return fileTransfer;
+        }
+
+        public long getLastUsed() {
+            return this.lastUsed;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
new file mode 100644
index 0000000..6387e19
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.standard.util.FileTransfer;
+import org.apache.nifi.processors.standard.util.SFTPTransfer;
+
+
+@SupportsBatching
+@Tags({"sftp", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"})
+@CapabilityDescription("Fetches the content of a file from a remote SFTP server and overwrites the contents of an incoming FlowFile with the content of the remote file.")
+@SeeAlso({GetSFTP.class, PutSFTP.class, GetFTP.class, PutFTP.class})
+@WritesAttributes({
+    @WritesAttribute(attribute = "sftp.remote.host", description = "The hostname or IP address from which the file was pulled"),
+    @WritesAttribute(attribute = "sftp.remote.port", description = "The port that was used to communicate with the remote SFTP server"),
+    @WritesAttribute(attribute = "sftp.remote.filename", description = "The name of the remote file that was pulled"),
+    @WritesAttribute(attribute = "filename", description = "The filename is updated to point to the filename fo the remote file"),
+})
+public class FetchSFTP extends FetchFileTransfer {
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(FetchFileTransfer.HOSTNAME);
+        properties.add(SFTPTransfer.PORT);
+        properties.add(SFTPTransfer.USERNAME);
+        properties.add(SFTPTransfer.PASSWORD);
+        properties.add(SFTPTransfer.PRIVATE_KEY_PATH);
+        properties.add(SFTPTransfer.PRIVATE_KEY_PASSPHRASE);
+        properties.add(FetchFileTransfer.REMOTE_FILENAME);
+        properties.add(SFTPTransfer.DELETE_ORIGINAL);
+        properties.add(SFTPTransfer.CONNECTION_TIMEOUT);
+        properties.add(SFTPTransfer.DATA_TIMEOUT);
+        properties.add(SFTPTransfer.USE_KEEPALIVE_ON_TIMEOUT);
+        properties.add(SFTPTransfer.HOST_KEY_FILE);
+        properties.add(SFTPTransfer.STRICT_HOST_KEY_CHECKING);
+        properties.add(SFTPTransfer.USE_COMPRESSION);
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        if (!validationContext.getProperty(SFTPTransfer.PASSWORD).isSet() && !(validationContext.getProperty(SFTPTransfer.PRIVATE_KEY_PASSPHRASE).isSet()
+            && validationContext.getProperty(SFTPTransfer.PRIVATE_KEY_PATH).isSet())) {
+            return Collections.singleton(new ValidationResult.Builder()
+                .subject("Password")
+                .valid(false)
+                .explanation("Must set either password or Private Key Path & Passphrase")
+                .build());
+        }
+
+        return Collections.emptyList();
+    }
+
+    @Override
+    protected FileTransfer createFileTransfer(final ProcessContext context) {
+        return new SFTPTransfer(context, getLogger());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
new file mode 100644
index 0000000..d6e1cd1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.util.FileInfo;
+import org.apache.nifi.processors.standard.util.FileTransfer;
+
+public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
+    public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
+        .name("Hostname")
+        .description("The fully qualified hostname or IP address of the remote system")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(true)
+        .build();
+    public static final PropertyDescriptor REMOTE_PATH = new PropertyDescriptor.Builder()
+        .name("Remote Path")
+        .description("The path on the remote system from which to pull or push files")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .defaultValue(".")
+        .build();
+
+
+    @Override
+    protected Map<String, String> createAttributes(final FileInfo fileInfo, final ProcessContext context) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(getProtocolName() + ".remote.host", context.getProperty(HOSTNAME).evaluateAttributeExpressions().getValue());
+        attributes.put("file.owner", fileInfo.getOwner());
+        attributes.put("file.group", fileInfo.getGroup());
+        attributes.put("file.permissions", fileInfo.getPermissions());
+        attributes.put(CoreAttributes.FILENAME.key(), fileInfo.getFileName());
+
+        final String fullPath = fileInfo.getFullPathFileName();
+        if (fullPath != null) {
+            final int index = fullPath.lastIndexOf("/");
+            if (index > -1) {
+                final String path = fullPath.substring(0, index);
+                attributes.put(CoreAttributes.PATH.key(), path);
+            }
+        }
+        return attributes;
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return context.getProperty(REMOTE_PATH).getValue();
+    }
+
+    @Override
+    protected List<FileInfo> performListing(final ProcessContext context, final Long minTimestamp) throws IOException {
+        final FileTransfer transfer = getFileTransfer(context);
+        final List<FileInfo> listing = transfer.getListing();
+        if (minTimestamp == null) {
+            return listing;
+        }
+
+        final Iterator<FileInfo> itr = listing.iterator();
+        while (itr.hasNext()) {
+            final FileInfo next = itr.next();
+            if (next.getLastModifiedTime() < minTimestamp) {
+                itr.remove();
+            }
+        }
+
+        return listing;
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(final PropertyDescriptor property) {
+        return HOSTNAME.equals(property) || REMOTE_PATH.equals(property);
+    }
+
+    protected abstract FileTransfer getFileTransfer(final ProcessContext context);
+
+    protected abstract String getProtocolName();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
new file mode 100644
index 0000000..3b6b69e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.components.PropertyDescriptor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.standard.util.FileTransfer;
+import org.apache.nifi.processors.standard.util.SFTPTransfer;
+
+@TriggerSerially
+@Tags({"list", "sftp", "remote", "ingest", "source", "input", "files"})
+@CapabilityDescription("Performs a listing of the files residing on an SFTP server. For each file that is found on the remote server, a new FlowFile will be created with the filename attribute "
+    + "set to the name of the file on the remote server. This can then be used in conjunction with FetchSFTP in order to fetch those files.")
+@SeeAlso({FetchSFTP.class, GetSFTP.class, PutSFTP.class})
+@WritesAttributes({
+    @WritesAttribute(attribute = "sftp.remote.host", description = "The hostname of the SFTP Server"),
+    @WritesAttribute(attribute = "file.owner", description = "The numeric owner id of the source file"),
+    @WritesAttribute(attribute = "file.group", description = "The numeric group id of the source file"),
+    @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the source file"),
+    @WritesAttribute(attribute = "filename", description = "The name of the file on the SFTP Server"),
+    @WritesAttribute(attribute = "path", description = "The fully qualified name of the directory on the SFTP Server from which the file was pulled"),
+})
+public class ListSFTP extends ListFileTransfer {
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(SFTPTransfer.HOSTNAME);
+        properties.add(SFTPTransfer.PORT);
+        properties.add(SFTPTransfer.USERNAME);
+        properties.add(SFTPTransfer.PASSWORD);
+        properties.add(SFTPTransfer.PRIVATE_KEY_PATH);
+        properties.add(SFTPTransfer.PRIVATE_KEY_PASSPHRASE);
+        properties.add(REMOTE_PATH);
+        properties.add(DISTRIBUTED_CACHE_SERVICE);
+        properties.add(SFTPTransfer.RECURSIVE_SEARCH);
+        properties.add(SFTPTransfer.FILE_FILTER_REGEX);
+        properties.add(SFTPTransfer.PATH_FILTER_REGEX);
+        properties.add(SFTPTransfer.IGNORE_DOTTED_FILES);
+        properties.add(SFTPTransfer.STRICT_HOST_KEY_CHECKING);
+        properties.add(SFTPTransfer.HOST_KEY_FILE);
+        properties.add(SFTPTransfer.CONNECTION_TIMEOUT);
+        properties.add(SFTPTransfer.DATA_TIMEOUT);
+        properties.add(SFTPTransfer.USE_KEEPALIVE_ON_TIMEOUT);
+        return properties;
+    }
+
+    @Override
+    protected FileTransfer getFileTransfer(final ProcessContext context) {
+        return new SFTPTransfer(context, getLogger());
+    }
+
+    @Override
+    protected String getProtocolName() {
+        return "sftp";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
new file mode 100644
index 0000000..56489f0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.standard.util;
+
+import java.util.Collection;
+import java.util.Date;
+
+import javax.xml.bind.annotation.XmlTransient;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * A simple POJO for maintaining state about the last entities listed by an AbstractListProcessor that was performed so that
+ * we can avoid pulling the same file multiple times
+ */
+@XmlType(name = "listing")
+public class EntityListing {
+
+    private Date latestTimestamp;
+    private Collection<String> matchingIdentifiers;
+
+    /**
+     * @return the modification date of the newest file that was contained in the listing
+     */
+    public Date getLatestTimestamp() {
+        return latestTimestamp;
+    }
+
+    /**
+     * Sets the timestamp of the modification date of the newest file that was contained in the listing
+     *
+     * @param latestTimestamp the timestamp of the modification date of the newest file that was contained in the listing
+     */
+    public void setLatestTimestamp(Date latestTimestamp) {
+        this.latestTimestamp = latestTimestamp;
+    }
+
+    /**
+     * @return a Collection containing the identifiers of all entities in the listing whose timestamp
+     *         was equal to {@link #getLatestTimestamp()}
+     */
+    @XmlTransient
+    public Collection<String> getMatchingIdentifiers() {
+        return matchingIdentifiers;
+    }
+
+    /**
+     * Sets the Collection containing the identifiers of all entities in the listing whose Timestamp was
+     * equal to {@link #getLatestTimestamp()}
+     * 
+     * @param matchingIdentifiers the identifiers that have last modified date matching the latest timestamp
+     */
+    public void setMatchingIdentifiers(Collection<String> matchingIdentifiers) {
+        this.matchingIdentifiers = matchingIdentifiers;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
index 41a42bb..7f659d4 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
@@ -34,16 +34,16 @@ import java.util.Locale;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import org.apache.commons.net.ftp.FTPClient;
+import org.apache.commons.net.ftp.FTPFile;
+import org.apache.commons.net.ftp.FTPHTTPClient;
+import org.apache.commons.net.ftp.FTPReply;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.commons.net.ftp.FTPClient;
-import org.apache.commons.net.ftp.FTPFile;
-import org.apache.commons.net.ftp.FTPHTTPClient;
-import org.apache.commons.net.ftp.FTPReply;
 
 public class FTPTransfer implements FileTransfer {
 
@@ -57,53 +57,53 @@ public class FTPTransfer implements FileTransfer {
     public static final String PROXY_TYPE_SOCKS = Proxy.Type.SOCKS.name();
 
     public static final PropertyDescriptor CONNECTION_MODE = new PropertyDescriptor.Builder()
-            .name("Connection Mode")
-            .description("The FTP Connection Mode")
-            .allowableValues(CONNECTION_MODE_ACTIVE, CONNECTION_MODE_PASSIVE)
-            .defaultValue(CONNECTION_MODE_PASSIVE)
-            .build();
+        .name("Connection Mode")
+        .description("The FTP Connection Mode")
+        .allowableValues(CONNECTION_MODE_ACTIVE, CONNECTION_MODE_PASSIVE)
+        .defaultValue(CONNECTION_MODE_PASSIVE)
+        .build();
     public static final PropertyDescriptor TRANSFER_MODE = new PropertyDescriptor.Builder()
-            .name("Transfer Mode")
-            .description("The FTP Transfer Mode")
-            .allowableValues(TRANSFER_MODE_BINARY, TRANSFER_MODE_ASCII)
-            .defaultValue(TRANSFER_MODE_BINARY)
-            .build();
+        .name("Transfer Mode")
+        .description("The FTP Transfer Mode")
+        .allowableValues(TRANSFER_MODE_BINARY, TRANSFER_MODE_ASCII)
+        .defaultValue(TRANSFER_MODE_BINARY)
+        .build();
     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
-            .name("Port")
-            .description("The port that the remote system is listening on for file transfers")
-            .addValidator(StandardValidators.PORT_VALIDATOR)
-            .required(true)
-            .defaultValue("21")
-            .build();
+        .name("Port")
+        .description("The port that the remote system is listening on for file transfers")
+        .addValidator(StandardValidators.PORT_VALIDATOR)
+        .required(true)
+        .defaultValue("21")
+        .build();
     public static final PropertyDescriptor PROXY_TYPE = new PropertyDescriptor.Builder()
-            .name("Proxy Type")
-            .description("Proxy type used for file transfers")
-            .allowableValues(PROXY_TYPE_DIRECT, PROXY_TYPE_HTTP, PROXY_TYPE_SOCKS)
-            .defaultValue(PROXY_TYPE_DIRECT)
-            .build();
+        .name("Proxy Type")
+        .description("Proxy type used for file transfers")
+        .allowableValues(PROXY_TYPE_DIRECT, PROXY_TYPE_HTTP, PROXY_TYPE_SOCKS)
+        .defaultValue(PROXY_TYPE_DIRECT)
+        .build();
     public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
-            .name("Proxy Host")
-            .description("The fully qualified hostname or IP address of the proxy server")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
+        .name("Proxy Host")
+        .description("The fully qualified hostname or IP address of the proxy server")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
     public static final PropertyDescriptor PROXY_PORT = new PropertyDescriptor.Builder()
-            .name("Proxy Port")
-            .description("The port of the proxy server")
-            .addValidator(StandardValidators.PORT_VALIDATOR)
-            .build();
+        .name("Proxy Port")
+        .description("The port of the proxy server")
+        .addValidator(StandardValidators.PORT_VALIDATOR)
+        .build();
     public static final PropertyDescriptor HTTP_PROXY_USERNAME = new PropertyDescriptor.Builder()
-            .name("Http Proxy Username")
-            .description("Http Proxy Username")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(false)
-            .build();
+        .name("Http Proxy Username")
+        .description("Http Proxy Username")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(false)
+        .build();
     public static final PropertyDescriptor HTTP_PROXY_PASSWORD = new PropertyDescriptor.Builder()
-            .name("Http Proxy Password")
-            .description("Http Proxy Password")
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .required(false)
-            .sensitive(true)
-            .build();
+        .name("Http Proxy Password")
+        .description("Http Proxy Password")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(false)
+        .sensitive(true)
+        .build();
 
     private final ProcessorLog logger;
 
@@ -135,7 +135,7 @@ public class FTPTransfer implements FileTransfer {
                 client.disconnect();
             }
         } catch (final Exception ex) {
-            logger.warn("Failed to close FTPClient due to {}", new Object[]{ex.toString()}, ex);
+            logger.warn("Failed to close FTPClient due to {}", new Object[] { ex.toString() }, ex);
         }
         client = null;
     }
@@ -261,19 +261,24 @@ public class FTPTransfer implements FileTransfer {
         perms.append(file.hasPermission(FTPFile.WORLD_ACCESS, FTPFile.EXECUTE_PERMISSION) ? "x" : "-");
 
         FileInfo.Builder builder = new FileInfo.Builder()
-                .filename(file.getName())
-                .fullPathFileName(newFullForwardPath)
-                .directory(file.isDirectory())
-                .size(file.getSize())
-                .lastModifiedTime(file.getTimestamp().getTimeInMillis())
-                .permissions(perms.toString())
-                .owner(file.getUser())
-                .group(file.getGroup());
+            .filename(file.getName())
+            .fullPathFileName(newFullForwardPath)
+            .directory(file.isDirectory())
+            .size(file.getSize())
+            .lastModifiedTime(file.getTimestamp().getTimeInMillis())
+            .permissions(perms.toString())
+            .owner(file.getUser())
+            .group(file.getGroup());
         return builder.build();
     }
 
     @Override
-    public InputStream getInputStream(final String remoteFileName) throws IOException {
+    public InputStream getInputStream(String remoteFileName) throws IOException {
+        return getInputStream(remoteFileName, null);
+    }
+
+    @Override
+    public InputStream getInputStream(final String remoteFileName, final FlowFile flowFile) throws IOException {
         final FTPClient client = getClient(null);
         InputStream in = client.retrieveFileStream(remoteFileName);
         if (in == null) {
@@ -329,9 +334,9 @@ public class FTPTransfer implements FileTransfer {
         final boolean cdSuccessful = setWorkingDirectory(remoteDirectory);
 
         if (!cdSuccessful) {
-            logger.debug("Remote Directory {} does not exist; creating it", new Object[]{remoteDirectory});
+            logger.debug("Remote Directory {} does not exist; creating it", new Object[] { remoteDirectory });
             if (client.makeDirectory(remoteDirectory)) {
-                logger.debug("Created {}", new Object[]{remoteDirectory});
+                logger.debug("Created {}", new Object[] { remoteDirectory });
             } else {
                 throw new IOException("Failed to create remote directory " + remoteDirectory);
             }
@@ -387,10 +392,10 @@ public class FTPTransfer implements FileTransfer {
                 final String time = outformat.format(fileModifyTime);
                 if (!client.setModificationTime(tempFilename, time)) {
                     // FTP server probably doesn't support MFMT command
-                    logger.warn("Could not set lastModifiedTime on {} to {}", new Object[]{flowFile, lastModifiedTime});
+                    logger.warn("Could not set lastModifiedTime on {} to {}", new Object[] { flowFile, lastModifiedTime });
                 }
             } catch (final Exception e) {
-                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[]{flowFile, lastModifiedTime, e});
+                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[] { flowFile, lastModifiedTime, e });
             }
         }
         final String permissions = ctx.getProperty(PERMISSIONS).evaluateAttributeExpressions(flowFile).getValue();
@@ -399,17 +404,17 @@ public class FTPTransfer implements FileTransfer {
                 int perms = numberPermissions(permissions);
                 if (perms >= 0) {
                     if (!client.sendSiteCommand("chmod " + Integer.toOctalString(perms) + " " + tempFilename)) {
-                        logger.warn("Could not set permission on {} to {}", new Object[]{flowFile, permissions});
+                        logger.warn("Could not set permission on {} to {}", new Object[] { flowFile, permissions });
                     }
                 }
             } catch (final Exception e) {
-                logger.error("Failed to set permission on {} to {} due to {}", new Object[]{flowFile, permissions, e});
+                logger.error("Failed to set permission on {} to {} due to {}", new Object[] { flowFile, permissions, e });
             }
         }
 
         if (!filename.equals(tempFilename)) {
             try {
-                logger.debug("Renaming remote path from {} to {} for {}", new Object[]{tempFilename, filename, flowFile});
+                logger.debug("Renaming remote path from {} to {} for {}", new Object[] { tempFilename, filename, flowFile });
                 final boolean renameSuccessful = client.rename(tempFilename, filename);
                 if (!renameSuccessful) {
                     throw new IOException("Failed to rename temporary file " + tempFilename + " to " + fullPath + " due to: " + client.getReplyString());
@@ -513,13 +518,13 @@ public class FTPTransfer implements FileTransfer {
             inetAddress = InetAddress.getByName(remoteHostname);
         }
 
-        client.connect(inetAddress, ctx.getProperty(PORT).asInteger());
+        client.connect(inetAddress, ctx.getProperty(PORT).evaluateAttributeExpressions(flowFile).asInteger());
         this.closed = false;
         client.setDataTimeout(ctx.getProperty(DATA_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue());
         client.setSoTimeout(ctx.getProperty(CONNECTION_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue());
 
-        final String username = ctx.getProperty(USERNAME).getValue();
-        final String password = ctx.getProperty(PASSWORD).getValue();
+        final String username = ctx.getProperty(USERNAME).evaluateAttributeExpressions(flowFile).getValue();
+        final String password = ctx.getProperty(PASSWORD).evaluateAttributeExpressions(flowFile).getValue();
         final boolean loggedIn = client.login(username, password);
         if (!loggedIn) {
             throw new IOException("Could not login for user '" + username + "'");
@@ -532,7 +537,7 @@ public class FTPTransfer implements FileTransfer {
             client.enterLocalPassiveMode();
         }
 
-        final String transferMode = ctx.getProperty(TRANSFER_MODE).getValue();
+        final String transferMode = ctx.getProperty(TRANSFER_MODE).evaluateAttributeExpressions(flowFile).getValue();
         final int fileType = (transferMode.equalsIgnoreCase(TRANSFER_MODE_ASCII)) ? FTPClient.ASCII_FILE_TYPE : FTPClient.BINARY_FILE_TYPE;
         if (!client.setFileType(fileType)) {
             throw new IOException("Unable to set transfer mode to type " + transferMode);

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1d57931/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
index c57b4e0..b893f75 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileInfo.java
@@ -18,7 +18,7 @@ package org.apache.nifi.processors.standard.util;
 
 import java.io.Serializable;
 
-public class FileInfo implements Comparable<FileInfo>, Serializable {
+public class FileInfo implements Comparable<FileInfo>, Serializable, ListableEntity {
 
     private static final long serialVersionUID = 1L;
 
@@ -164,4 +164,20 @@ public class FileInfo implements Comparable<FileInfo>, Serializable {
             return this;
         }
     }
+
+    @Override
+    public String getName() {
+        return getFileName();
+    }
+
+    @Override
+    public String getIdentifier() {
+        final String fullPathName = getFullPathFileName();
+        return fullPathName == null ? getName() : fullPathName;
+    }
+
+    @Override
+    public long getTimestamp() {
+        return getLastModifiedTime();
+    }
 }


[46/50] [abbrv] nifi git commit: NIFI-1051: Fixed checkstyle violations

Posted by ma...@apache.org.
NIFI-1051: Fixed checkstyle violations


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

Branch: refs/heads/NIFI-730
Commit: abf2c6288458958be54a7a9a5fddc91693aaa4cf
Parents: 7a73867
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Nov 2 11:01:50 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 11:01:50 2015 -0500

----------------------------------------------------------------------
 .../repository/FileSystemRepository.java        | 12 ++++-----
 .../repository/TestFileSystemRepository.java    | 26 ++++++++++----------
 2 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/abf2c628/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 1354b4a..b4a1716 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -309,12 +309,12 @@ public class FileSystemRepository implements ContentRepository {
                     // the path already exists, so scan the path to find any files and update maxIndex to the max of
                     // all filenames seen.
                     Files.walkFileTree(realPath, new SimpleFileVisitor<Path>() {
-                    	
-						public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
-							LOG.warn("Content repository contains un-readable file or directory '" + file.getFileName() + "'. Skipping. ", exc);
-							return FileVisitResult.SKIP_SUBTREE;
-						}
-                    		 
+                        @Override
+                        public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
+                            LOG.warn("Content repository contains un-readable file or directory '" + file.getFileName() + "'. Skipping. ", exc);
+                            return FileVisitResult.SKIP_SUBTREE;
+                        }
+
                         @Override
                         public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs) throws IOException {
                             if (attrs.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/abf2c628/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 95e1f40..5da67a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -71,23 +71,23 @@ public class TestFileSystemRepository {
     public void shutdown() throws IOException {
         repository.shutdown();
     }
-    
+
     @Test
     public void testBogusFile() throws IOException {
-    	repository.shutdown();
-    	System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
-        
+        repository.shutdown();
+        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+
         File bogus = new File(rootFile, "bogus");
         try {
-        	 bogus.mkdir();
-             bogus.setReadable(false);
-        
-             repository = new FileSystemRepository();
-             repository.initialize(new StandardResourceClaimManager());
-		} finally {
-			bogus.setReadable(true);
-			assertTrue(bogus.delete());
-		}
+            bogus.mkdir();
+            bogus.setReadable(false);
+
+            repository = new FileSystemRepository();
+            repository.initialize(new StandardResourceClaimManager());
+        } finally {
+            bogus.setReadable(true);
+            assertTrue(bogus.delete());
+        }
     }
 
     @Test


[18/50] [abbrv] nifi git commit: NIFI-447 adding replacement strategy to ReplaceText processor

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextLineByLine.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextLineByLine.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextLineByLine.java
deleted file mode 100644
index 005c05a..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceTextLineByLine.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.processors.standard;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestReplaceTextLineByLine {
-
-    @Test
-    public void testSimple() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "odo");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "ood");
-
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/food.txt")));
-    }
-
-    @Test
-    public void testBackReference() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(DODO)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
-
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/[DODO].txt")));
-    }
-
-    @Test
-    public void testReplacementWithExpressionLanguageIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(jo)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[${abc}]");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "$1");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt")));
-    }
-
-    @Test
-    public void testRegexWithExpressionLanguage() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "${replaceKey}");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("replaceKey", "Riley");
-        attributes.put("replaceValue", "Spider");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Spider.txt")));
-    }
-
-    @Test
-    public void testRegexWithExpressionLanguageIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "${replaceKey}");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("replaceKey", "R.*y");
-        attributes.put("replaceValue", "Spider");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-    }
-
-    @Test
-    public void testBackReferenceWithTooLargeOfIndexIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(lu)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1$2");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("replaceKey", "R.*y");
-        attributes.put("replaceValue", "Spiderman");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt")));
-    }
-
-    @Test
-    public void testBackReferenceWithInvalidReferenceIsEscaped() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(ew)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$d");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("replaceKey", "H.*o");
-        attributes.put("replaceValue", "Good-bye");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt")));
-    }
-
-    @Test
-    public void testEscapingDollarSign() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(DO)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\\$1");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("replaceKey", "H.*o");
-        attributes.put("replaceValue", "Good-bye");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/$1$1.txt")));
-    }
-
-    @Test
-    public void testReplaceWithEmptyString() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(jo)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "");
-
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/cu_Po.txt")));
-    }
-
-    @Test
-    public void testWithNoMatch() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "Z");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "Morning");
-
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-    }
-
-    @Test
-    public void testWithMultipleMatches() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "l");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "R");
-
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt")));
-    }
-
-    @Test
-    public void testAttributeToContent() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, ".*");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "Good");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Good.txt")));
-    }
-
-    @Test
-    public void testAttributeToContentWindows() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, ".*");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("abc", "Good");
-        runner.enqueue("<<<HEADER>>>\r\n<<BODY>>\r\n<<<FOOTER>>>\r".getBytes(), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        out.assertContentEquals("GoodGoodGood");
-    }
-
-    @Test
-    public void testProblematicCase1() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, ".*");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${filename}\t${now():format(\"yyyy/MM/dd'T'HHmmss'Z'\")}\t${fileSize}\n");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "abc.txt");
-        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        final String outContent = translateNewLines(new String(out.toByteArray(), StandardCharsets.UTF_8));
-        Assert.assertTrue(outContent.startsWith("abc.txt\t"));
-        System.out.println(outContent);
-        Assert.assertTrue(outContent.endsWith("193\n") || outContent.endsWith("203\r\n"));
-    }
-
-    @Test
-    public void testGetExistingContent() throws IOException {
-        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
-        runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
-        runner.setProperty(ReplaceText.REGEX, "(?s)(^.*)");
-        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "attribute header\n\n${filename}\n\ndata header\n\n$1\n\nfooter\n");
-
-        final Map<String, String> attributes = new HashMap<>();
-        attributes.put("filename", "abc.txt");
-        runner.enqueue("Hello\nWorld!".getBytes(), attributes);
-
-        runner.run();
-
-        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
-        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
-        final String outContent = new String(out.toByteArray(), StandardCharsets.UTF_8);
-        System.out.println(outContent);
-        Assert.assertTrue(outContent.equals("attribute header\n\nabc.txt\n\ndata header\n\nHello\n\n\nfooter\n"
-                + "attribute header\n\nabc.txt\n\ndata header\n\nWorld!\n\nfooter\n"));
-
-    }
-
-    private byte[] translateNewLines(final File file) throws IOException {
-        return translateNewLines(file.toPath());
-    }
-
-    private byte[] translateNewLines(final Path path) throws IOException {
-        final byte[] data = Files.readAllBytes(path);
-        final String text = new String(data, StandardCharsets.UTF_8);
-        return translateNewLines(text).getBytes(StandardCharsets.UTF_8);
-    }
-
-    private String translateNewLines(final String text) {
-        final String lineSeparator = System.getProperty("line.separator");
-        final Pattern pattern = Pattern.compile("\n", Pattern.MULTILINE);
-        final String translated = pattern.matcher(text).replaceAll(lineSeparator);
-        return translated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt
new file mode 100644
index 0000000..56a0d55
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt
@@ -0,0 +1,11 @@
+<<<HEADER>>> TEST
+Fodo DODO cujo Pojo TEST
+Blue Dew clue hew TEST
+Grampa Riley Huey TEST
+Fodo DODO cujo Pojo TEST
+Blue Dew clue hew TEST
+Grampa Riley Huey TEST
+Fodo DODO cujo Pojo TEST
+Blue Dew clue hew TEST
+Grampa Riley Huey TEST
+<<<FOOTER>>> TEST
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt
new file mode 100644
index 0000000..ffc0d77
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt
@@ -0,0 +1,11 @@
+TEST <<<HEADER>>>
+TEST Fodo DODO cujo Pojo
+TEST Blue Dew clue hew
+TEST Grampa Riley Huey
+TEST Fodo DODO cujo Pojo
+TEST Blue Dew clue hew
+TEST Grampa Riley Huey
+TEST Fodo DODO cujo Pojo
+TEST Blue Dew clue hew
+TEST Grampa Riley Huey
+TEST <<<FOOTER>>>
\ No newline at end of file


[43/50] [abbrv] nifi git commit: NIFI-1068 Fix EndpointConnectionPool to properly remove connections from activeConnections when terminating connections

Posted by ma...@apache.org.
NIFI-1068 Fix EndpointConnectionPool to properly remove connections from activeConnections when terminating connections

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: 37e2f178f8f0e4a0fed022e2541a64e97e4897d4
Parents: ad849c7
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Mon Oct 26 17:35:23 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Nov 1 14:47:23 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/remote/client/socket/EndpointConnectionPool.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/37e2f178/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index b162b3f..88a34aa 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -820,12 +820,13 @@ public class EndpointConnectionPool {
         for (final BlockingQueue<EndpointConnection> connectionQueue : connectionQueueMap.values()) {
             EndpointConnection state;
             while ((state = connectionQueue.poll()) != null) {
-                cleanup(state.getSocketClientProtocol(), state.getPeer());
+                terminate(state);
             }
         }
     }
 
     public void terminate(final EndpointConnection connection) {
+        activeConnections.remove(connection);
         cleanup(connection.getSocketClientProtocol(), connection.getPeer());
     }
 


[15/50] [abbrv] nifi git commit: NIFI-10: Fixed checkstyle violation

Posted by ma...@apache.org.
NIFI-10: Fixed checkstyle violation


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

Branch: refs/heads/NIFI-730
Commit: 17006335e5b687a8a268d1571ae04a98235e6394
Parents: fc2aa27
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Oct 26 17:09:51 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 17:09:51 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/provenance/ProvenanceReporter.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/17006335/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
index 0fd29fd..39eed43 100644
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
+++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceReporter.java
@@ -159,7 +159,7 @@ public interface ProvenanceReporter {
      * the data
      */
     void fetch(FlowFile flowFile, String transitUri, String details, long transmissionMillis);
-    
+
     /**
      * Emits a Provenance Event of type {@link ProvenanceEventType#SEND SEND}
      * that indicates that a copy of the given FlowFile was sent to an external


[48/50] [abbrv] nifi git commit: NIFI-1077 enabling expression language on ConvertCharacterSet input and output

Posted by ma...@apache.org.
NIFI-1077 enabling expression language on ConvertCharacterSet input and output

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: a549621267c6517c4774579c3d95f0d7a0c1f68b
Parents: 814e8b2
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Mon Nov 2 11:30:06 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 11:46:17 2015 -0500

----------------------------------------------------------------------
 .../nifi/processor/util/StandardValidators.java | 10 +++
 .../standard/ConvertCharacterSet.java           | 19 +++---
 .../standard/TestConvertCharacterSet.java       | 64 +++++++++++++++++++-
 3 files changed, 84 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/a5496212/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
index 7777438..8255781 100644
--- a/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
+++ b/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java
@@ -209,6 +209,16 @@ public class StandardValidators {
         @Override
         public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
             if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value)) {
+                final ResultType resultType = context.newExpressionLanguageCompiler().getResultType(value);
+                if (!resultType.equals(ResultType.STRING)) {
+                    return new ValidationResult.Builder()
+                            .subject(subject)
+                            .input(value)
+                            .valid(false)
+                            .explanation("Expected Attribute Query to return type " + ResultType.STRING + " but query returns type " + resultType)
+                            .build();
+                }
+
                 return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build();
             }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/a5496212/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
index 7a99a59..53c1954 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessorInitializationContext;
@@ -87,12 +88,14 @@ public class ConvertCharacterSet extends AbstractProcessor {
     public static final PropertyDescriptor INPUT_CHARSET = new PropertyDescriptor.Builder()
             .name("Input Character Set")
             .description("The name of the CharacterSet to expect for Input")
+            .expressionLanguageSupported(true)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();
     public static final PropertyDescriptor OUTPUT_CHARSET = new PropertyDescriptor.Builder()
             .name("Output Character Set")
             .description("The name of the CharacterSet to convert to")
+            .expressionLanguageSupported(true)
             .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
             .required(true)
             .build();
@@ -128,10 +131,15 @@ public class ConvertCharacterSet extends AbstractProcessor {
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
         final ProcessorLog logger = getLogger();
 
-        final Charset inputCharset = Charset.forName(context.getProperty(INPUT_CHARSET).getValue());
-        final Charset outputCharset = Charset.forName(context.getProperty(OUTPUT_CHARSET).getValue());
+        final Charset inputCharset = Charset.forName(context.getProperty(INPUT_CHARSET).evaluateAttributeExpressions(flowFile).getValue());
+        final Charset outputCharset = Charset.forName(context.getProperty(OUTPUT_CHARSET).evaluateAttributeExpressions(flowFile).getValue());
         final CharBuffer charBuffer = CharBuffer.allocate(MAX_BUFFER_SIZE);
 
         final CharsetDecoder decoder = inputCharset.newDecoder();
@@ -144,11 +152,6 @@ public class ConvertCharacterSet extends AbstractProcessor {
         encoder.onUnmappableCharacter(CodingErrorAction.REPLACE);
         encoder.replaceWith("?".getBytes(outputCharset));
 
-        FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return;
-        }
-
         try {
             final StopWatch stopWatch = new StopWatch(true);
             flowFile = session.write(flowFile, new StreamCallback() {
@@ -169,7 +172,7 @@ public class ConvertCharacterSet extends AbstractProcessor {
 
             session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
             logger.info("successfully converted characters from {} to {} for {}",
-                    new Object[]{context.getProperty(INPUT_CHARSET).getValue(), context.getProperty(OUTPUT_CHARSET).getValue(), flowFile});
+                    new Object[]{inputCharset, outputCharset, flowFile});
             session.transfer(flowFile, REL_SUCCESS);
         } catch (final Exception e) {
             throw new ProcessException(e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/a5496212/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java
index 1b057d9..fea0a4e 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertCharacterSet.java
@@ -19,16 +19,20 @@ package org.apache.nifi.processors.standard;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
 
+import static junit.framework.TestCase.fail;
+
 public class TestConvertCharacterSet {
 
     @Test
-    public void test() throws IOException {
+    public void testSimple() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ConvertCharacterSet());
         runner.setProperty(ConvertCharacterSet.INPUT_CHARSET, "ASCII");
         runner.setProperty(ConvertCharacterSet.OUTPUT_CHARSET, "UTF-32");
@@ -41,4 +45,62 @@ public class TestConvertCharacterSet {
         output.assertContentEquals(new File("src/test/resources/CharacterSetConversionSamples/Converted2.txt"));
     }
 
+    @Test
+    public void testExpressionLanguageInput() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ConvertCharacterSet());
+        runner.setProperty(ConvertCharacterSet.INPUT_CHARSET, "${characterSet}");
+        runner.setProperty(ConvertCharacterSet.OUTPUT_CHARSET, "UTF-32");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("characterSet", "ASCII");
+        runner.enqueue(Paths.get("src/test/resources/CharacterSetConversionSamples/Original.txt"),attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ConvertCharacterSet.REL_SUCCESS, 1);
+        final MockFlowFile output = runner.getFlowFilesForRelationship(ConvertCharacterSet.REL_SUCCESS).get(0);
+        output.assertContentEquals(new File("src/test/resources/CharacterSetConversionSamples/Converted2.txt"));
+    }
+
+    @Test
+    public void testExpressionLanguageOutput() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ConvertCharacterSet());
+        runner.setProperty(ConvertCharacterSet.INPUT_CHARSET, "ASCII");
+        runner.setProperty(ConvertCharacterSet.OUTPUT_CHARSET, "${characterSet}");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("characterSet", "UTF-32");
+        runner.enqueue(Paths.get("src/test/resources/CharacterSetConversionSamples/Original.txt"),attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ConvertCharacterSet.REL_SUCCESS, 1);
+        final MockFlowFile output = runner.getFlowFilesForRelationship(ConvertCharacterSet.REL_SUCCESS).get(0);
+        output.assertContentEquals(new File("src/test/resources/CharacterSetConversionSamples/Converted2.txt"));
+    }
+
+    @Test
+    public void testExpressionLanguageConfig() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ConvertCharacterSet());
+        runner.setProperty(ConvertCharacterSet.INPUT_CHARSET, "${now()}");
+        runner.setProperty(ConvertCharacterSet.OUTPUT_CHARSET, "UTF-32");
+
+        runner.enqueue(Paths.get("src/test/resources/CharacterSetConversionSamples/Original.txt"));
+        try {
+            runner.run();
+            fail("Should fail to validate config and fail to run the on trigger");
+        } catch (AssertionError e){
+            // Expect to fail assertion for passing a date to the character set validator
+        }
+
+
+        runner.setProperty(ConvertCharacterSet.INPUT_CHARSET, "UTF-32");
+        runner.setProperty(ConvertCharacterSet.OUTPUT_CHARSET, "${anyAttribute(\"abc\", \"xyz\"):contains(\"bye\")}");
+
+        runner.enqueue(Paths.get("src/test/resources/CharacterSetConversionSamples/Original.txt"));
+        try {
+            runner.run();
+            fail("Should fail to validate config and fail to run the on trigger");
+        } catch (AssertionError e) {
+            // Expect to fail assertion for passing a boolean to the character set validator
+        }
+    }
 }


[05/50] [abbrv] nifi git commit: NIFI-673: Added Completion Strategy to FetchSFTP

Posted by ma...@apache.org.
NIFI-673: Added Completion Strategy to FetchSFTP


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

Branch: refs/heads/NIFI-730
Commit: b0322d9ffe8d117aae4faf7dd3e2881a28940f96
Parents: d1d5793
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Oct 5 16:11:40 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:13:02 2015 -0400

----------------------------------------------------------------------
 .../standard/AbstractListProcessor.java         |  38 +++---
 .../processors/standard/FetchFileTransfer.java  |  66 ++++++++--
 .../nifi/processors/standard/FetchSFTP.java     |  15 ++-
 .../processors/standard/ListFileTransfer.java   |   8 ++
 .../nifi/processors/standard/ListSFTP.java      |   7 +-
 .../processors/standard/util/EntityListing.java |   2 +-
 .../processors/standard/util/FTPTransfer.java   |  26 ++--
 .../processors/standard/util/FileTransfer.java  |   2 +
 .../processors/standard/util/SFTPTransfer.java  |  17 +++
 .../standard/TestFetchFileTransfer.java         | 131 +++++++++++++++++++
 10 files changed, 265 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
index 8a7fade..e592483 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractListProcessor.java
@@ -70,23 +70,25 @@ import org.codehaus.jackson.map.ObjectMapper;
  *
  * <p>
  * In order to make use of this abstract class, the entities listed must meet the following criteria:
- * <ul>
- * <li>
- * Entity must have a timestamp associated with it. This timestamp is used to determine if entities are "new" or not. Any entity that is
- * returned by the listing will be considered "new" if the timestamp is later than the latest timestamp pulled.
- * </li>
- * <li>
- * Entity must have a unique identifier. This is used in conjunction with the timestamp in order to determine whether or not the entity is
- * new. If the timestamp of an entity is before the latest timestamp pulled, then the entity is not considered new. If the timestamp is later
- * than the last timestamp pulled, then the entity is considered new. If the timestamp is equal to the latest timestamp pulled, then the entity's
- * identifier is compared to all of the entity identifiers that have that same timestamp in order to determine whether or not the entity has been
- * seen already.
- * </li>
- * <li>
- * Entity must have a user-readable name that can be used for logging purposes.
- * </li>
  * </p>
  *
+ * <ul>
+ *  <li>
+ *      Entity must have a timestamp associated with it. This timestamp is used to determine if entities are "new" or not. Any entity that is
+ *      returned by the listing will be considered "new" if the timestamp is later than the latest timestamp pulled.
+ *  </li>
+ *  <li>
+ *      Entity must have a unique identifier. This is used in conjunction with the timestamp in order to determine whether or not the entity is
+ *      new. If the timestamp of an entity is before the latest timestamp pulled, then the entity is not considered new. If the timestamp is later
+ *      than the last timestamp pulled, then the entity is considered new. If the timestamp is equal to the latest timestamp pulled, then the entity's
+ *      identifier is compared to all of the entity identifiers that have that same timestamp in order to determine whether or not the entity has been
+ *      seen already.
+ *  </li>
+ *  <li>
+ *      Entity must have a user-readable name that can be used for logging purposes.
+ *  </li>
+ * </ul>
+ *
  * <p>
  * This class persists state across restarts so that even if NiFi is restarted, duplicates will not be pulled from the remote system. This is performed using
  * two different mechanisms. First, state is stored locally. This allows the system to be restarted and begin processing where it left off. The state that is
@@ -111,6 +113,7 @@ import org.codehaus.jackson.map.ObjectMapper;
  *
  * <p>
  * Subclasses are responsible for the following:
+ * </p>
  *
  * <ul>
  * <li>
@@ -134,7 +137,6 @@ import org.codehaus.jackson.map.ObjectMapper;
  * a boolean indicating whether or not a change in the value of the provided property should trigger the timestamp and identifier information to be cleared.
  * </li>
  * </ul>
- * </p>
  */
 @TriggerSerially
 public abstract class AbstractListProcessor<T extends ListableEntity> extends AbstractProcessor {
@@ -372,8 +374,8 @@ public abstract class AbstractListProcessor<T extends ListableEntity> extends Ab
         int listCount = 0;
         Long latestListingTimestamp = null;
         for (final T entity : entityList) {
-            final boolean list = (minTimestamp == null || entity.getTimestamp() > minTimestamp ||
-                (entity.getTimestamp() == minTimestamp && !latestIdentifiersListed.contains(entity.getIdentifier())));
+            final boolean list = (minTimestamp == null || entity.getTimestamp() > minTimestamp
+                || (entity.getTimestamp() == minTimestamp && !latestIdentifiersListed.contains(entity.getIdentifier())));
 
             // Create the FlowFile for this path.
             if (list) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
index 5eecac3..ab0be78 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -31,7 +31,9 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -49,9 +51,18 @@ import org.apache.nifi.util.StopWatch;
 import org.apache.nifi.util.Tuple;
 
 /**
- * A base class for FetchSFTP, FetchFTP processors
+ * A base class for FetchSFTP, FetchFTP processors.
+ *
+ * Note that implementations of this class should never use the @SupportsBatching annotation! Doing so
+ * could result in data loss!
  */
 public abstract class FetchFileTransfer extends AbstractProcessor {
+
+    static final AllowableValue COMPLETION_NONE = new AllowableValue("None", "None", "Leave the file as-is");
+    static final AllowableValue COMPLETION_MOVE = new AllowableValue("Move File", "Move File", "Move the file to the directory specified by the <Move Destination Directory> property");
+    static final AllowableValue COMPLETION_DELETE = new AllowableValue("Delete File", "Delete File", "Deletes the original file from the remote system");
+
+
     static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder()
         .name("Hostname")
         .description("The fully-qualified hostname or IP address of the host to fetch the data from")
@@ -73,13 +84,25 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
         .expressionLanguageSupported(true)
         .build();
-    public static final PropertyDescriptor DELETE_ORIGINAL = new PropertyDescriptor.Builder()
-        .name("Delete Original")
-        .description("Determines whether or not the file is deleted from the remote system after it has been successfully transferred")
-        .defaultValue("true")
-        .allowableValues("true", "false")
+    static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder()
+        .name("Completion Strategy")
+        .description("Specifies what to do with the original file on the server once it has been pulled into NiFi. If the Completion Strategy fails, a warning will be "
+            + "logged but the data will still be transferred.")
+        .expressionLanguageSupported(false)
+        .allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE)
+        .defaultValue(COMPLETION_NONE.getValue())
         .required(true)
         .build();
+    static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder()
+        .name("Move Destination Directory")
+        .description("The directory on the remote server to the move the original file to once it has been ingested into NiFi. "
+            + "This property is ignored unless the Completion Strategy is set to \"Move File\". The specified directory must already exist on"
+            + "the remote system, or the rename will fail.")
+        .expressionLanguageSupported(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(false)
+        .build();
+
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
         .name("success")
@@ -156,7 +179,8 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         properties.add(HOSTNAME);
         properties.add(UNDEFAULTED_PORT);
         properties.add(REMOTE_FILENAME);
-        properties.add(DELETE_ORIGINAL);
+        properties.add(COMPLETION_STRATEGY);
+        properties.add(MOVE_DESTINATION_DIR);
         return properties;
     }
 
@@ -203,6 +227,7 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         final InputStream in;
         try {
             in = transfer.getInputStream(filename, flowFile);
+
             flowFile = session.write(flowFile, new OutputStreamCallback() {
                 @Override
                 public void process(final OutputStream out) throws IOException {
@@ -250,15 +275,34 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
             stopWatch.getElapsed(TimeUnit.MILLISECONDS));
         session.transfer(flowFile, REL_SUCCESS);
 
-        // delete remote file is necessary
-        final boolean deleteOriginal = context.getProperty(DELETE_ORIGINAL).asBoolean();
-        if (deleteOriginal) {
+        // it is critical that we commit the session before moving/deleting the remote file. Otherwise, we could have a situation where
+        // we ingest the data, delete/move the remote file, and then NiFi dies/is shut down before the session is committed. This would
+        // result in data loss! If we commit the session first, we are safe.
+        session.commit();
+
+        final String completionStrategy = context.getProperty(COMPLETION_STRATEGY).getValue();
+        if (COMPLETION_DELETE.getValue().equalsIgnoreCase(completionStrategy)) {
             try {
                 transfer.deleteFile(null, filename);
             } catch (final FileNotFoundException e) {
                 // file doesn't exist -- effectively the same as removing it. Move on.
             } catch (final IOException ioe) {
-                getLogger().warn("Successfully fetched the content for {} from {}:{}{} but failed to remove the remote file due to {}", new Object[] {flowFile, host, port, filename, ioe}, ioe);
+                getLogger().warn("Successfully fetched the content for {} from {}:{}{} but failed to remove the remote file due to {}",
+                    new Object[] {flowFile, host, port, filename, ioe}, ioe);
+            }
+        } else if (COMPLETION_MOVE.getValue().equalsIgnoreCase(completionStrategy)) {
+            String targetDir = context.getProperty(MOVE_DESTINATION_DIR).evaluateAttributeExpressions(flowFile).getValue();
+            if (!targetDir.endsWith("/")) {
+                targetDir = targetDir + "/";
+            }
+            final String simpleFilename = StringUtils.substringAfterLast(filename, "/");
+            final String target = targetDir + simpleFilename;
+
+            try {
+                transfer.rename(filename, target);
+            } catch (final IOException ioe) {
+                getLogger().warn("Successfully fetched the content for {} from {}:{}{} but failed to rename the remote file due to {}",
+                    new Object[] {flowFile, host, port, filename, ioe}, ioe);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
index 6387e19..ad81c83 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -35,8 +34,7 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processors.standard.util.FileTransfer;
 import org.apache.nifi.processors.standard.util.SFTPTransfer;
 
-
-@SupportsBatching
+// Note that we do not use @SupportsBatching annotation. This processor cannot support batching because it must ensure that session commits happen before remote files are deleted.
 @Tags({"sftp", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"})
 @CapabilityDescription("Fetches the content of a file from a remote SFTP server and overwrites the contents of an incoming FlowFile with the content of the remote file.")
 @SeeAlso({GetSFTP.class, PutSFTP.class, GetFTP.class, PutFTP.class})
@@ -50,15 +48,18 @@ public class FetchSFTP extends FetchFileTransfer {
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final PropertyDescriptor port = new PropertyDescriptor.Builder().fromPropertyDescriptor(UNDEFAULTED_PORT).defaultValue("22").build();
+
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(FetchFileTransfer.HOSTNAME);
-        properties.add(SFTPTransfer.PORT);
+        properties.add(HOSTNAME);
+        properties.add(port);
         properties.add(SFTPTransfer.USERNAME);
         properties.add(SFTPTransfer.PASSWORD);
         properties.add(SFTPTransfer.PRIVATE_KEY_PATH);
         properties.add(SFTPTransfer.PRIVATE_KEY_PASSPHRASE);
-        properties.add(FetchFileTransfer.REMOTE_FILENAME);
-        properties.add(SFTPTransfer.DELETE_ORIGINAL);
+        properties.add(REMOTE_FILENAME);
+        properties.add(COMPLETION_STRATEGY);
+        properties.add(MOVE_DESTINATION_DIR);
         properties.add(SFTPTransfer.CONNECTION_TIMEOUT);
         properties.add(SFTPTransfer.DATA_TIMEOUT);
         properties.add(SFTPTransfer.USE_KEEPALIVE_ON_TIMEOUT);

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
index d6e1cd1..1176fd0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
@@ -38,6 +38,13 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
         .required(true)
         .expressionLanguageSupported(true)
         .build();
+    static final PropertyDescriptor UNDEFAULTED_PORT = new PropertyDescriptor.Builder()
+        .name("Port")
+        .description("The port to connect to on the remote host to fetch the data from")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
     public static final PropertyDescriptor REMOTE_PATH = new PropertyDescriptor.Builder()
         .name("Remote Path")
         .description("The path on the remote system from which to pull or push files")
@@ -52,6 +59,7 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
     protected Map<String, String> createAttributes(final FileInfo fileInfo, final ProcessContext context) {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put(getProtocolName() + ".remote.host", context.getProperty(HOSTNAME).evaluateAttributeExpressions().getValue());
+        attributes.put(getProtocolName() + ".remote.port", context.getProperty(UNDEFAULTED_PORT).evaluateAttributeExpressions().getValue());
         attributes.put("file.owner", fileInfo.getOwner());
         attributes.put("file.group", fileInfo.getGroup());
         attributes.put("file.permissions", fileInfo.getPermissions());

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
index 3b6b69e..925e5f8 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
@@ -38,6 +38,7 @@ import org.apache.nifi.processors.standard.util.SFTPTransfer;
 @SeeAlso({FetchSFTP.class, GetSFTP.class, PutSFTP.class})
 @WritesAttributes({
     @WritesAttribute(attribute = "sftp.remote.host", description = "The hostname of the SFTP Server"),
+    @WritesAttribute(attribute = "sftp.remote.port", description = "The port that was connected to on the SFTP Server"),
     @WritesAttribute(attribute = "file.owner", description = "The numeric owner id of the source file"),
     @WritesAttribute(attribute = "file.group", description = "The numeric group id of the source file"),
     @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the source file"),
@@ -48,9 +49,11 @@ public class ListSFTP extends ListFileTransfer {
 
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final PropertyDescriptor port = new PropertyDescriptor.Builder().fromPropertyDescriptor(UNDEFAULTED_PORT).defaultValue("22").build();
+
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(SFTPTransfer.HOSTNAME);
-        properties.add(SFTPTransfer.PORT);
+        properties.add(HOSTNAME);
+        properties.add(port);
         properties.add(SFTPTransfer.USERNAME);
         properties.add(SFTPTransfer.PASSWORD);
         properties.add(SFTPTransfer.PRIVATE_KEY_PATH);

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
index 56489f0..2d9525f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/EntityListing.java
@@ -61,7 +61,7 @@ public class EntityListing {
     /**
      * Sets the Collection containing the identifiers of all entities in the listing whose Timestamp was
      * equal to {@link #getLatestTimestamp()}
-     * 
+     *
      * @param matchingIdentifiers the identifiers that have last modified date matching the latest timestamp
      */
     public void setMatchingIdentifiers(Collection<String> matchingIdentifiers) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
index 7f659d4..a038eb7 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FTPTransfer.java
@@ -135,7 +135,7 @@ public class FTPTransfer implements FileTransfer {
                 client.disconnect();
             }
         } catch (final Exception ex) {
-            logger.warn("Failed to close FTPClient due to {}", new Object[] { ex.toString() }, ex);
+            logger.warn("Failed to close FTPClient due to {}", new Object[] {ex.toString()}, ex);
         }
         client = null;
     }
@@ -334,9 +334,9 @@ public class FTPTransfer implements FileTransfer {
         final boolean cdSuccessful = setWorkingDirectory(remoteDirectory);
 
         if (!cdSuccessful) {
-            logger.debug("Remote Directory {} does not exist; creating it", new Object[] { remoteDirectory });
+            logger.debug("Remote Directory {} does not exist; creating it", new Object[] {remoteDirectory});
             if (client.makeDirectory(remoteDirectory)) {
-                logger.debug("Created {}", new Object[] { remoteDirectory });
+                logger.debug("Created {}", new Object[] {remoteDirectory});
             } else {
                 throw new IOException("Failed to create remote directory " + remoteDirectory);
             }
@@ -392,10 +392,10 @@ public class FTPTransfer implements FileTransfer {
                 final String time = outformat.format(fileModifyTime);
                 if (!client.setModificationTime(tempFilename, time)) {
                     // FTP server probably doesn't support MFMT command
-                    logger.warn("Could not set lastModifiedTime on {} to {}", new Object[] { flowFile, lastModifiedTime });
+                    logger.warn("Could not set lastModifiedTime on {} to {}", new Object[] {flowFile, lastModifiedTime});
                 }
             } catch (final Exception e) {
-                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[] { flowFile, lastModifiedTime, e });
+                logger.error("Failed to set lastModifiedTime on {} to {} due to {}", new Object[] {flowFile, lastModifiedTime, e});
             }
         }
         final String permissions = ctx.getProperty(PERMISSIONS).evaluateAttributeExpressions(flowFile).getValue();
@@ -404,17 +404,17 @@ public class FTPTransfer implements FileTransfer {
                 int perms = numberPermissions(permissions);
                 if (perms >= 0) {
                     if (!client.sendSiteCommand("chmod " + Integer.toOctalString(perms) + " " + tempFilename)) {
-                        logger.warn("Could not set permission on {} to {}", new Object[] { flowFile, permissions });
+                        logger.warn("Could not set permission on {} to {}", new Object[] {flowFile, permissions});
                     }
                 }
             } catch (final Exception e) {
-                logger.error("Failed to set permission on {} to {} due to {}", new Object[] { flowFile, permissions, e });
+                logger.error("Failed to set permission on {} to {} due to {}", new Object[] {flowFile, permissions, e});
             }
         }
 
         if (!filename.equals(tempFilename)) {
             try {
-                logger.debug("Renaming remote path from {} to {} for {}", new Object[] { tempFilename, filename, flowFile });
+                logger.debug("Renaming remote path from {} to {} for {}", new Object[] {tempFilename, filename, flowFile});
                 final boolean renameSuccessful = client.rename(tempFilename, filename);
                 if (!renameSuccessful) {
                     throw new IOException("Failed to rename temporary file " + tempFilename + " to " + fullPath + " due to: " + client.getReplyString());
@@ -432,6 +432,16 @@ public class FTPTransfer implements FileTransfer {
         return fullPath;
     }
 
+
+    @Override
+    public void rename(final String source, final String target) throws IOException {
+        final FTPClient client = getClient(null);
+        final boolean renameSuccessful = client.rename(source, target);
+        if (!renameSuccessful) {
+            throw new IOException("Failed to rename temporary file " + source + " to " + target + " due to: " + client.getReplyString());
+        }
+    }
+
     @Override
     public void deleteFile(final String path, final String remoteFileName) throws IOException {
         final FTPClient client = getClient(null);

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
index fe277df..8d48de2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/FileTransfer.java
@@ -43,6 +43,8 @@ public interface FileTransfer extends Closeable {
 
     String put(FlowFile flowFile, String path, String filename, InputStream content) throws IOException;
 
+    void rename(String source, String target) throws IOException;
+
     void deleteFile(String path, String remoteFileName) throws IOException;
 
     void deleteDirectory(String remoteDirectoryName) throws IOException;

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
index c28f275..9bad520 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/SFTPTransfer.java
@@ -603,6 +603,23 @@ public class SFTPTransfer implements FileTransfer {
         return fullPath;
     }
 
+    @Override
+    public void rename(final String source, final String target) throws IOException {
+        final ChannelSftp sftp = getChannel(null);
+        try {
+            sftp.rename(source, target);
+        } catch (final SftpException e) {
+            switch (e.id) {
+                case ChannelSftp.SSH_FX_NO_SUCH_FILE:
+                    throw new FileNotFoundException();
+                case ChannelSftp.SSH_FX_PERMISSION_DENIED:
+                    throw new PermissionDeniedException("Could not rename remote file " + source + " to " + target + " due to insufficient permissions");
+                default:
+                    throw new IOException(e);
+            }
+        }
+    }
+
     protected int numberPermissions(String perms) {
         int number = -1;
         final Pattern rwxPattern = Pattern.compile("^[rwx-]{9}$");

http://git-wip-us.apache.org/repos/asf/nifi/blob/b0322d9f/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
index 7aa8f9c..4175a77 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestFetchFileTransfer.java
@@ -17,7 +17,9 @@
 
 package org.apache.nifi.processors.standard;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
@@ -92,8 +94,119 @@ public class TestFetchFileTransfer {
         runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_PERMISSION_DENIED, 1);
     }
 
+
+    @Test
+    public void testMoveFileWithNoTrailingSlashDirName() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+        runner.setProperty(FetchFileTransfer.COMPLETION_STRATEGY, FetchFileTransfer.COMPLETION_MOVE.getValue());
+        runner.setProperty(FetchFileTransfer.MOVE_DESTINATION_DIR, "/moved");
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+
+        proc.fileContents.containsKey("/moved/hello.txt");
+        assertEquals(1, proc.fileContents.size());
+    }
+
+    @Test
+    public void testMoveFileWithTrailingSlashDirName() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+        runner.setProperty(FetchFileTransfer.COMPLETION_STRATEGY, FetchFileTransfer.COMPLETION_MOVE.getValue());
+        runner.setProperty(FetchFileTransfer.MOVE_DESTINATION_DIR, "/moved/");
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+
+        proc.fileContents.containsKey("/moved/hello.txt");
+        assertEquals(1, proc.fileContents.size());
+    }
+
+    @Test
+    public void testDeleteFile() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+        runner.setProperty(FetchFileTransfer.COMPLETION_STRATEGY, FetchFileTransfer.COMPLETION_DELETE.getValue());
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+        assertTrue(proc.fileContents.isEmpty());
+    }
+
+    @Test
+    public void testDeleteFails() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+        runner.setProperty(FetchFileTransfer.COMPLETION_STRATEGY, FetchFileTransfer.COMPLETION_DELETE.getValue());
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+        proc.allowDelete = false;
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+        assertFalse(proc.fileContents.isEmpty());
+    }
+
+    @Test
+    public void testRenameFails() {
+        final TestableFetchFileTransfer proc = new TestableFetchFileTransfer();
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(FetchFileTransfer.HOSTNAME, "localhost");
+        runner.setProperty(FetchFileTransfer.UNDEFAULTED_PORT, "11");
+        runner.setProperty(FetchFileTransfer.REMOTE_FILENAME, "${filename}");
+        runner.setProperty(FetchFileTransfer.COMPLETION_STRATEGY, FetchFileTransfer.COMPLETION_MOVE.getValue());
+        runner.setProperty(FetchFileTransfer.MOVE_DESTINATION_DIR, "/moved/");
+
+        proc.addContent("hello.txt", "world".getBytes());
+        final Map<String, String> attrs = new HashMap<>();
+        attrs.put("filename", "hello.txt");
+        runner.enqueue(new byte[0], attrs);
+        proc.allowDelete = false;
+        proc.allowRename = false;
+
+        runner.run(1, false, false);
+        runner.assertAllFlowFilesTransferred(FetchFileTransfer.REL_SUCCESS, 1);
+        assertEquals(1, proc.fileContents.size());
+
+        assertTrue(proc.fileContents.containsKey("hello.txt"));
+    }
+
+
     private static class TestableFetchFileTransfer extends FetchFileTransfer {
         private boolean allowAccess = true;
+        private boolean allowDelete = true;
+        private boolean allowRename = true;
         private boolean closed = false;
         private final Map<String, byte[]> fileContents = new HashMap<>();
 
@@ -154,6 +267,10 @@ public class TestFetchFileTransfer {
 
                 @Override
                 public void deleteFile(String path, String remoteFileName) throws IOException {
+                    if (!allowDelete) {
+                        throw new PermissionDeniedException("test permission denied");
+                    }
+
                     if (!fileContents.containsKey(remoteFileName)) {
                         throw new FileNotFoundException();
                     }
@@ -162,6 +279,20 @@ public class TestFetchFileTransfer {
                 }
 
                 @Override
+                public void rename(String source, String target) throws IOException {
+                    if (!allowRename) {
+                        throw new PermissionDeniedException("test permission denied");
+                    }
+
+                    if (!fileContents.containsKey(source)) {
+                        throw new FileNotFoundException();
+                    }
+
+                    final byte[] content = fileContents.remove(source);
+                    fileContents.put(target, content);
+                }
+
+                @Override
                 public void deleteDirectory(String remoteDirectoryName) throws IOException {
 
                 }


[19/50] [abbrv] nifi git commit: NIFI-447 adding replacement strategy to ReplaceText processor

Posted by ma...@apache.org.
NIFI-447 adding replacement strategy to ReplaceText processor

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: 07238c79137ac393441e2f4a0d0fdfdf26ebd277
Parents: 1c17386
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Mon Oct 26 23:15:31 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Oct 27 10:37:43 2015 -0400

----------------------------------------------------------------------
 .../nifi-standard-processors/pom.xml            |   5 +-
 .../nifi/processors/standard/ReplaceText.java   | 431 +++++++++----
 .../processors/standard/TestReplaceText.java    | 635 ++++++++++++++++++-
 .../standard/TestReplaceTextLineByLine.java     | 336 ----------
 .../AppendLineByLineTest.txt                    |  11 +
 .../PrependLineByLineTest.txt                   |  11 +
 6 files changed, 953 insertions(+), 476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index b0b3afa..4ee382f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -190,15 +190,12 @@ language governing permissions and limitations under the License. -->
             <artifactId>derby</artifactId>
             <scope>test</scope>
         </dependency>
-        
         <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
             <version>1.4.187</version>
             <scope>test</scope>
         </dependency>
-              
-        
     </dependencies>
     
     <build>
@@ -251,6 +248,8 @@ language governing permissions and limitations under the License. -->
                         <exclude>src/test/resources/TestReplaceTextLineByLine/cu_Po.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/food.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/testFile.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt</exclude>

http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
index 98a56bf..fb51d45 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
@@ -35,10 +35,12 @@ import java.util.regex.Pattern;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.commons.io.IOUtils;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.expression.AttributeValueDecorator;
@@ -64,84 +66,111 @@ import org.apache.nifi.util.StopWatch;
 @SupportsBatching
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex"})
-@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of "
-        + "the content that matches the Regular Expression with some alternate value.")
+@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression (regex) against it and replacing the section of "
+    + "the content that matches the Regular Expression with some alternate value.")
 public class ReplaceText extends AbstractProcessor {
 
-    //Constants
+    // Constants
     public static final String LINE_BY_LINE = "Line-by-Line";
     public static final String ENTIRE_TEXT = "Entire text";
+    public static final String prependValue = "Prepend";
+    public static final String appendValue = "Append";
+    public static final String regexReplaceValue = "Regex Replace";
+    public static final String literalReplaceValue = "Literal Replace";
     private final Pattern backReferencePattern = Pattern.compile("\\$(\\d+)");
-    private static final byte[] ZERO_BYTE_BUFFER = new byte[0];
     private static final String DEFAULT_REGEX = "(?s:^.*$)";
     private static final String DEFAULT_REPLACEMENT_VALUE = "$1";
 
-    // Properties
-    public static final PropertyDescriptor REGEX = new PropertyDescriptor.Builder()
-            .name("Regular Expression")
-            .description("The Regular Expression to search for in the FlowFile content")
-            .required(true)
-            .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
-            .expressionLanguageSupported(true)
-            .defaultValue(DEFAULT_REGEX)
-            .build();
+    // Prepend and Append will just insert the replacement value at the beginning or end
+    // Properties PREPEND, APPEND, REGEX_REPLACE, LITERAL_REPLACE
+    static final AllowableValue PREPEND = new AllowableValue(prependValue, prependValue,
+        "Insert the Replacement Value at the beginning of the FlowFile or the beginning of each line (depending on the Evaluation Mode). For \"Line-by-Line\" Evaluation Mode, "
+            + "the value will be prepended to each line. For \"Entire Text\" evaluation mode, the value will be prepended to the entire text.");
+    static final AllowableValue APPEND = new AllowableValue(appendValue, appendValue,
+        "Insert the Replacement Value at the end of the FlowFile or the end of each line (depending on the Evluation Mode). For \"Line-by-Line\" Evaluation Mode, "
+            + "the value will be appended to each line. For \"Entire Text\" evaluation mode, the value will be appended to the entire text.");
+    static final AllowableValue LITERAL_REPLACE = new AllowableValue(literalReplaceValue, literalReplaceValue,
+        "Search for all instances of the Search Value and replace the matches with the Replacement Value.");
+    static final AllowableValue REGEX_REPLACE = new AllowableValue(regexReplaceValue, regexReplaceValue,
+        "Interpret the Search Value as a Regular Expression and replace all matches with the Replacement Value. The Replacement Value may reference Capturing Groups used "
+            + "in the Search Value by using a dollar-sign followed by the Capturing Group number, such as $1 or $2. If the Search Value is set to .* then everything is replaced without "
+            + "even evaluating the Regular Expression.");
+
+    public static final PropertyDescriptor SEARCH_VALUE = new PropertyDescriptor.Builder()
+        .name("Regular Expression")
+        .displayName("Search Value")
+        .description("The Search Value to search for in the FlowFile content. Only used for 'Literal Replace' and 'Regex Replace' matching strategies")
+        .required(true)
+        .addValidator(StandardValidators.createRegexValidator(0, Integer.MAX_VALUE, true))
+        .expressionLanguageSupported(true)
+        .defaultValue(DEFAULT_REGEX)
+        .build();
     public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder()
-            .name("Replacement Value")
-            .description("The value to replace the regular expression with. Back-references to Regular Expression capturing groups are supported, but "
-                    + "back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.")
-            .required(true)
-            .defaultValue(DEFAULT_REPLACEMENT_VALUE)
-            .addValidator(Validator.VALID)
-            .expressionLanguageSupported(true)
-            .build();
+        .name("Replacement Value")
+        .description("The value to insert using the 'Replacement Strategy'. Using \"Regex Replace\" back-references to Regular Expression capturing groups "
+            + "are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.")
+        .required(true)
+        .defaultValue(DEFAULT_REPLACEMENT_VALUE)
+        .addValidator(Validator.VALID)
+        .expressionLanguageSupported(true)
+        .build();
     public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
-            .name("Character Set")
-            .description("The Character Set in which the file is encoded")
-            .required(true)
-            .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
-            .defaultValue("UTF-8")
-            .build();
+        .name("Character Set")
+        .description("The Character Set in which the file is encoded")
+        .required(true)
+        .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
+        .defaultValue("UTF-8")
+        .build();
     public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
-            .name("Maximum Buffer Size")
-            .description("Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to "
-                    + "apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, "
-                    + "the FlowFile will be routed to 'failure'. "
-                    + "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value "
-                    + "of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. "
-                    + "This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'")
-            .required(true)
-            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
-            .defaultValue("1 MB")
-            .build();
+        .name("Maximum Buffer Size")
+        .description("Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to "
+            + "apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, "
+            + "the FlowFile will be routed to 'failure'. "
+            + "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value "
+            + "of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. "
+            + "This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'")
+        .required(true)
+        .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+        .defaultValue("1 MB")
+        .build();
+    public static final PropertyDescriptor REPLACEMENT_STRATEGY = new PropertyDescriptor.Builder()
+        .name("Replacement Strategy")
+        .description("The strategy for how and what to replace within the FlowFile's text content.")
+        .allowableValues(PREPEND, APPEND, REGEX_REPLACE, LITERAL_REPLACE)
+        .defaultValue(REGEX_REPLACE.getValue())
+        .required(true)
+        .build();
     public static final PropertyDescriptor EVALUATION_MODE = new PropertyDescriptor.Builder()
-            .name("Evaluation Mode")
-            .description("Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and "
-                    + "then evaluate the 'Regular Expression'.")
-            .allowableValues(LINE_BY_LINE, ENTIRE_TEXT)
-            .defaultValue(ENTIRE_TEXT)
-            .required(true)
-            .build();
+        .name("Evaluation Mode")
+        .description("Run the 'Replacement Strategy' against each line separately (Line-by-Line) or buffer the entire file into memory (Entire Text) "
+            + "and run against that.")
+        .allowableValues(LINE_BY_LINE, ENTIRE_TEXT)
+        .defaultValue(ENTIRE_TEXT)
+        .required(true)
+        .build();
+
     // Relationships
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
-            .name("success")
-            .description("FlowFiles that have been successfully updated are routed to this relationship, as well as FlowFiles whose content does not "
-                    + "match the given Regular Expression")
-            .build();
+        .name("success")
+        .description("FlowFiles that have been successfully processed are routed to this relationship. This includes both FlowFiles that had text"
+            + " replaced and those that did not.")
+        .build();
     public static final Relationship REL_FAILURE = new Relationship.Builder()
-            .name("failure")
-            .description("FlowFiles that could not be updated are routed to this relationship")
-            .build();
-    //
+        .name("failure")
+        .description("FlowFiles that could not be updated are routed to this relationship")
+        .build();
+
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
 
     @Override
     protected void init(final ProcessorInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.add(REGEX);
+        properties.add(SEARCH_VALUE);
         properties.add(REPLACEMENT_VALUE);
         properties.add(CHARACTER_SET);
         properties.add(MAX_BUFFER_SIZE);
+        properties.add(REPLACEMENT_STRATEGY);
         properties.add(EVALUATION_MODE);
         this.properties = Collections.unmodifiableList(properties);
 
@@ -169,22 +198,18 @@ public class ReplaceText extends AbstractProcessor {
         }
 
         final ProcessorLog logger = getLogger();
-        final String unsubstitutedRegex = context.getProperty(REGEX).getValue();
+
+        final String unsubstitutedRegex = context.getProperty(SEARCH_VALUE).getValue();
         String unsubstitutedReplacement = context.getProperty(REPLACEMENT_VALUE).getValue();
-        if (unsubstitutedRegex.equals(DEFAULT_REGEX) && unsubstitutedReplacement.equals(DEFAULT_REPLACEMENT_VALUE)) {
+        final String replacementStrategy = context.getProperty(REPLACEMENT_STRATEGY).getValue();
+
+        if (replacementStrategy.equalsIgnoreCase(regexReplaceValue) && unsubstitutedRegex.equals(DEFAULT_REGEX) && unsubstitutedReplacement.equals(DEFAULT_REPLACEMENT_VALUE)) {
             // This pattern says replace content with itself. We can highly optimize this process by simply transferring
             // all FlowFiles to the 'success' relationship
             session.transfer(flowFiles, REL_SUCCESS);
             return;
         }
 
-        final AttributeValueDecorator quotedAttributeDecorator = new AttributeValueDecorator() {
-            @Override
-            public String decorate(final String attributeValue) {
-                return Pattern.quote(attributeValue);
-            }
-        };
-
         final AttributeValueDecorator escapeBackRefDecorator = new AttributeValueDecorator() {
             @Override
             public String decorate(final String attributeValue) {
@@ -192,7 +217,7 @@ public class ReplaceText extends AbstractProcessor {
             }
         };
 
-        final String regexValue = context.getProperty(REGEX).evaluateAttributeExpressions().getValue();
+        final String regexValue = context.getProperty(SEARCH_VALUE).evaluateAttributeExpressions().getValue();
         final int numCapturingGroups = Pattern.compile(regexValue).matcher("").groupCount();
 
         final boolean skipBuffer = ".*".equals(unsubstitutedRegex);
@@ -200,9 +225,13 @@ public class ReplaceText extends AbstractProcessor {
         final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
         final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
 
-        final byte[] buffer = skipBuffer ? ZERO_BYTE_BUFFER : new byte[maxBufferSize];
-
         final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
+        final byte[] buffer;
+        if (replacementStrategy.equalsIgnoreCase(regexReplaceValue) || replacementStrategy.equalsIgnoreCase(literalReplaceValue)) {
+            buffer = new byte[maxBufferSize];
+        } else {
+            buffer = null;
+        }
 
         for (FlowFile flowFile : flowFiles) {
             if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
@@ -212,47 +241,175 @@ public class ReplaceText extends AbstractProcessor {
                 }
             }
 
-            String replacement = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile, escapeBackRefDecorator).getValue();
-            final Matcher backRefMatcher = backReferencePattern.matcher(replacement);
-            while (backRefMatcher.find()) {
-                final String backRefNum = backRefMatcher.group(1);
-                if (backRefNum.startsWith("0")) {
-                    continue;
-                }
-                final int originalBackRefIndex = Integer.parseInt(backRefNum);
-                int backRefIndex = originalBackRefIndex;
-
-                // if we have a replacement value like $123, and we have less than 123 capturing groups, then
-                // we want to truncate the 3 and use capturing group 12; if we have less than 12 capturing groups,
-                // then we want to truncate the 2 and use capturing group 1; if we don't have a capturing group then
-                // we want to truncate the 1 and get 0.
-                while (backRefIndex > numCapturingGroups && backRefIndex >= 10) {
-                    backRefIndex /= 10;
-                }
+            String replacement;
+            if (!replacementStrategy.equals(regexReplaceValue)) {
+                replacement = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
+            } else {
+                replacement = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile, escapeBackRefDecorator).getValue();
+                final Matcher backRefMatcher = backReferencePattern.matcher(replacement);
+                while (backRefMatcher.find()) {
+                    final String backRefNum = backRefMatcher.group(1);
+                    if (backRefNum.startsWith("0")) {
+                        continue;
+                    }
+                    final int originalBackRefIndex = Integer.parseInt(backRefNum);
+                    int backRefIndex = originalBackRefIndex;
+
+                    // if we have a replacement value like $123, and we have less than 123 capturing groups, then
+                    // we want to truncate the 3 and use capturing group 12; if we have less than 12 capturing groups,
+                    // then we want to truncate the 2 and use capturing group 1; if we don't have a capturing group then
+                    // we want to truncate the 1 and get 0.
+                    while (backRefIndex > numCapturingGroups && backRefIndex >= 10) {
+                        backRefIndex /= 10;
+                    }
 
-                if (backRefIndex > numCapturingGroups) {
-                    final StringBuilder sb = new StringBuilder(replacement.length() + 1);
-                    final int groupStart = backRefMatcher.start(1);
+                    if (backRefIndex > numCapturingGroups) {
+                        final StringBuilder sb = new StringBuilder(replacement.length() + 1);
+                        final int groupStart = backRefMatcher.start(1);
 
-                    sb.append(replacement.substring(0, groupStart - 1));
-                    sb.append("\\");
-                    sb.append(replacement.substring(groupStart - 1));
-                    replacement = sb.toString();
+                        sb.append(replacement.substring(0, groupStart - 1));
+                        sb.append("\\");
+                        sb.append(replacement.substring(groupStart - 1));
+                        replacement = sb.toString();
+                    }
                 }
             }
 
-            replacement = replacement.replaceAll("(\\$\\D)", "\\\\$1");
+            ReplacementStrategyExecutor replacementStrategyExecutor;
+            switch (replacementStrategy) {
+                case prependValue:
+                    replacementStrategyExecutor = new PrependReplace();
+                    break;
+                case appendValue:
+                    replacementStrategyExecutor = new AppendReplace();
+                    break;
+                case regexReplaceValue:
+                    replacementStrategyExecutor = new RegexReplace(buffer);
+                    break;
+                case literalReplaceValue:
+                    replacementStrategyExecutor = new LiteralReplace(buffer);
+                    break;
+                default:
+                    throw new AssertionError();
+            }
+
+            final StopWatch stopWatch = new StopWatch(true);
+
+            flowFile = replacementStrategyExecutor.replace(flowFile, session, context, replacement, evaluateMode,
+                charset, maxBufferSize, skipBuffer);
+
+            logger.info("Transferred {} to 'success'", new Object[] {flowFile});
+            session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+    }
+
+    private static class PrependReplace implements ReplacementStrategyExecutor {
+
+        @Override
+        public FlowFile replace(FlowFile flowFile, final ProcessSession session, final ProcessContext context, final String replacementValue, final String evaluateMode,
+            final Charset charset, final int maxBufferSize, final boolean skipBuffer) {
+            if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
+                flowFile = session.write(flowFile, new StreamCallback() {
+                    @Override
+                    public void process(final InputStream in, final OutputStream out) throws IOException {
+                        out.write(replacementValue.getBytes(charset));
+                        IOUtils.copy(in, out);
+                    }
+                });
+            } else {
+                flowFile = session.write(flowFile, new StreamCallback() {
+                    @Override
+                    public void process(final InputStream in, final OutputStream out) throws IOException {
+                        try (NLKBufferedReader br = new NLKBufferedReader(new InputStreamReader(in, charset), maxBufferSize);
+                            BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset));) {
+                            String oneLine;
+                            while (null != (oneLine = br.readLine())) {
+                                final String updatedValue = replacementValue.concat(oneLine);
+                                bw.write(updatedValue);
+                            }
+                        }
+                    }
+                });
+            }
+            return flowFile;
+        }
+    }
+
+    private static class AppendReplace implements ReplacementStrategyExecutor {
+
+        @Override
+        public FlowFile replace(FlowFile flowFile, final ProcessSession session, final ProcessContext context, final String replacementValue, final String evaluateMode,
+            final Charset charset, final int maxBufferSize, final boolean skipBuffer) {
+            if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
+                flowFile = session.write(flowFile, new StreamCallback() {
+                    @Override
+                    public void process(final InputStream in, final OutputStream out) throws IOException {
+                        IOUtils.copy(in, out);
+                        out.write(replacementValue.getBytes(charset));
+                    }
+                });
+            } else {
+                flowFile = session.write(flowFile, new StreamCallback() {
+                    @Override
+                    public void process(final InputStream in, final OutputStream out) throws IOException {
+                        try (NLKBufferedReader br = new NLKBufferedReader(new InputStreamReader(in, charset), maxBufferSize);
+                            BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset));) {
+                            String oneLine;
+                            while (null != (oneLine = br.readLine())) {
+                                // we need to find the first carriage return or new-line so that we can append the new value
+                                // before the line separate. However, we don't want to do this using a regular expression due
+                                // to performance concerns. So we will find the first occurrence of either \r or \n and use
+                                // that to insert the replacement value.
+                                boolean foundNewLine = false;
+                                for (int i = 0; i < oneLine.length(); i++) {
+                                    final char c = oneLine.charAt(i);
+                                    if (foundNewLine) {
+                                        bw.write(c);
+                                        continue;
+                                    }
+
+                                    if (c == '\r' || c == '\n') {
+                                        bw.write(replacementValue);
+                                        foundNewLine = true;
+                                    }
+
+                                    bw.write(c);
+                                }
+
+                                if (!foundNewLine) {
+                                    bw.write(replacementValue);
+                                }
+                            }
+                        }
+                    }
+                });
+            }
+            return flowFile;
+        }
+    }
+
+
+    private static class RegexReplace implements ReplacementStrategyExecutor {
+        private final byte[] buffer;
+
+        public RegexReplace(final byte[] buffer) {
+            this.buffer = buffer;
+        }
+
+        @Override
+        public FlowFile replace(FlowFile flowFile, final ProcessSession session, final ProcessContext context, final String replacementValue, final String evaluateMode,
+            final Charset charset, final int maxBufferSize, final boolean skipBuffer) {
+            final String replacementFinal = replacementValue.replaceAll("(\\$\\D)", "\\\\$1");
 
             // always match; just overwrite value with the replacement value; this optimization prevents us
             // from reading the file at all.
-            final String replacementValue = replacement;
             if (skipBuffer) {
-                final StopWatch stopWatch = new StopWatch(true);
                 if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
                     flowFile = session.write(flowFile, new OutputStreamCallback() {
                         @Override
                         public void process(final OutputStream out) throws IOException {
-                            out.write(replacementValue.getBytes(charset));
+                            out.write(replacementFinal.getBytes(charset));
                         }
                     });
                 } else {
@@ -260,31 +417,83 @@ public class ReplaceText extends AbstractProcessor {
                         @Override
                         public void process(final InputStream in, final OutputStream out) throws IOException {
                             try (NLKBufferedReader br = new NLKBufferedReader(new InputStreamReader(in, charset), maxBufferSize);
-                                    BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset));) {
+                                BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset));) {
                                 while (null != br.readLine()) {
-                                    bw.write(replacementValue);
+                                    bw.write(replacementFinal);
+                                }
+                            }
+                        }
+                    });
+                }
+            } else {
+                final AttributeValueDecorator quotedAttributeDecorator = new AttributeValueDecorator() {
+                    @Override
+                    public String decorate(final String attributeValue) {
+                        return Pattern.quote(attributeValue);
+                    }
+                };
+                final String searchRegex = context.getProperty(SEARCH_VALUE).evaluateAttributeExpressions(flowFile, quotedAttributeDecorator).getValue();
+
+                final int flowFileSize = (int) flowFile.getSize();
+                if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
+                    flowFile = session.write(flowFile, new StreamCallback() {
+                        @Override
+                        public void process(final InputStream in, final OutputStream out) throws IOException {
+                            StreamUtils.fillBuffer(in, buffer, false);
+                            final String contentString = new String(buffer, 0, flowFileSize, charset);
+                            final String updatedValue = contentString.replaceAll(searchRegex, replacementFinal);
+                            out.write(updatedValue.getBytes(charset));
+                        }
+                    });
+                } else {
+                    flowFile = session.write(flowFile, new StreamCallback() {
+                        @Override
+                        public void process(final InputStream in, final OutputStream out) throws IOException {
+                            try (NLKBufferedReader br = new NLKBufferedReader(new InputStreamReader(in, charset), maxBufferSize);
+                                BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset))) {
+                                String oneLine;
+                                while (null != (oneLine = br.readLine())) {
+                                    final String updatedValue = oneLine.replaceAll(searchRegex, replacementFinal);
+                                    bw.write(updatedValue);
                                 }
                             }
                         }
                     });
                 }
-                session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
-                session.transfer(flowFile, REL_SUCCESS);
-                logger.info("Transferred {} to 'success'", new Object[]{flowFile});
-                continue;
             }
+            return flowFile;
+        }
+    }
+
+    private static class LiteralReplace implements ReplacementStrategyExecutor {
+        private final byte[] buffer;
+
+        public LiteralReplace(final byte[] buffer) {
+            this.buffer = buffer;
+        }
+
+        @Override
+        public FlowFile replace(FlowFile flowFile, final ProcessSession session, final ProcessContext context, final String replacementValue, final String evaluateMode,
+            final Charset charset, final int maxBufferSize, final boolean skipBuffer) {
+            final AttributeValueDecorator quotedAttributeDecorator = new AttributeValueDecorator() {
+                @Override
+                public String decorate(final String attributeValue) {
+                    return Pattern.quote(attributeValue);
+                }
+            };
+
+            final String searchValue = context.getProperty(SEARCH_VALUE).evaluateAttributeExpressions(flowFile, quotedAttributeDecorator).getValue();
 
-            final StopWatch stopWatch = new StopWatch(true);
-            final String regex = context.getProperty(REGEX).evaluateAttributeExpressions(flowFile, quotedAttributeDecorator).getValue();
 
+            final int flowFileSize = (int) flowFile.getSize();
             if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
-                final int flowFileSize = (int) flowFile.getSize();
                 flowFile = session.write(flowFile, new StreamCallback() {
                     @Override
                     public void process(final InputStream in, final OutputStream out) throws IOException {
                         StreamUtils.fillBuffer(in, buffer, false);
                         final String contentString = new String(buffer, 0, flowFileSize, charset);
-                        final String updatedValue = contentString.replaceAll(regex, replacementValue);
+                        // Interpreting the search and replacement values as char sequences
+                        final String updatedValue = contentString.replace(searchValue, replacementValue);
                         out.write(updatedValue.getBytes(charset));
                     }
                 });
@@ -293,20 +502,22 @@ public class ReplaceText extends AbstractProcessor {
                     @Override
                     public void process(final InputStream in, final OutputStream out) throws IOException {
                         try (NLKBufferedReader br = new NLKBufferedReader(new InputStreamReader(in, charset), maxBufferSize);
-                                BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset));) {
+                            BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(out, charset))) {
                             String oneLine;
                             while (null != (oneLine = br.readLine())) {
-                                final String updatedValue = oneLine.replaceAll(regex, replacementValue);
+                                // Interpreting the search and replacement values as char sequences
+                                final String updatedValue = oneLine.replace(searchValue, replacementValue);
                                 bw.write(updatedValue);
                             }
                         }
                     }
                 });
             }
-
-            logger.info("Transferred {} to 'success'", new Object[]{flowFile});
-            session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
-            session.transfer(flowFile, REL_SUCCESS);
+            return flowFile;
         }
     }
+
+    private interface ReplacementStrategyExecutor {
+        FlowFile replace(FlowFile flowFile, ProcessSession session, ProcessContext context, String replacement, String evaluateMode, Charset charset, int maxBufferSize, boolean skipBuffer);
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/07238c79/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
index ab5f6be..3a311a3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java
@@ -16,25 +16,43 @@
  */
 package org.apache.nifi.processors.standard;
 
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.junit.Assert;
-import org.junit.Test;
-
+import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
 
 public class TestReplaceText {
 
     @Test
+    public void testConfigurationCornerCase() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+
+        runner.run();
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(Paths.get("src/test/resources/hello.txt"));
+    }
+
+    @Test
     public void testSimple() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "ell");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "lle");
 
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
@@ -46,10 +64,156 @@ public class TestReplaceText {
     }
 
     @Test
+    public void testPrependSimple() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
+
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("TESTHello, World!".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testPrependLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "_");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+
+        runner.enqueue("hello\nthere\nmadam".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("_hello\n_there\n_madam".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testAppendSimple() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Hello, World!TEST".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testAppendWithCarriageReturn() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+
+        runner.enqueue("hello\rthere\rsir".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("hello!\rthere!\rsir!");
+    }
+
+    @Test
+    public void testAppendWithNewLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+
+        runner.enqueue("hello\nthere\nsir".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("hello!\nthere!\nsir!");
+    }
+
+    @Test
+    public void testAppendWithCarriageReturnNewLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "!");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+
+        runner.enqueue("hello\r\nthere\r\nsir".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("hello!\r\nthere!\r\nsir!");
+    }
+
+    @Test
+    public void testLiteralSimple() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "lle");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Hlleo, World!".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testLiteralBackReference() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("H[$1]o, World!");
+    }
+
+    @Test
+    public void testLiteral() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".ell.");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "test");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue(Paths.get("src/test/resources/hello.txt"));
+        runner.run();
+        runner.enqueue("H.ell.o, World! .ell.".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 2);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Hello, World!");
+        final MockFlowFile out2 = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(1);
+        out2.assertContentEquals("Htesto, World! test");
+    }
+
+    @Test
     public void testBackReference() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
 
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
@@ -65,7 +229,7 @@ public class TestReplaceText {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
         String expected = "Hell23o, World!";
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$123");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -83,7 +247,7 @@ public class TestReplaceText {
     public void testBackRefWithNoCapturingGroup() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "ell");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "ell");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$0123");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -97,10 +261,10 @@ public class TestReplaceText {
     }
 
     @Test
-    public void testAmy3() throws IOException {
+    public void testReplacementWithExpressionLanguage() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "${replaceKey}");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "GoodBye");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -118,7 +282,7 @@ public class TestReplaceText {
     public void testReplacementWithExpressionLanguageIsEscaped() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[${abc}]");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -136,7 +300,7 @@ public class TestReplaceText {
     public void testRegexWithExpressionLanguage() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "${replaceKey}");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -155,7 +319,7 @@ public class TestReplaceText {
     public void testRegexWithExpressionLanguageIsEscaped() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "${replaceKey}");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -174,7 +338,7 @@ public class TestReplaceText {
     public void testBackReferenceWithTooLargeOfIndexIsEscaped() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1$2");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -193,7 +357,7 @@ public class TestReplaceText {
     public void testBackReferenceWithInvalidReferenceIsEscaped() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$d");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -212,7 +376,7 @@ public class TestReplaceText {
     public void testEscapingDollarSign() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\\$1");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -231,7 +395,7 @@ public class TestReplaceText {
     public void testReplaceWithEmptyString() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(ell)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ell)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "");
 
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
@@ -246,7 +410,7 @@ public class TestReplaceText {
     public void testWithNoMatch() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "Z");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "Z");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "Morning");
 
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
@@ -261,7 +425,7 @@ public class TestReplaceText {
     public void testWithMultipleMatches() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "l");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "l");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "R");
 
         runner.enqueue(Paths.get("src/test/resources/hello.txt"));
@@ -276,7 +440,7 @@ public class TestReplaceText {
     public void testAttributeToContent() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, ".*");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -294,7 +458,7 @@ public class TestReplaceText {
     public void testRoutesToFailureIfTooLarge() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "[123]");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "[123]");
         runner.setProperty(ReplaceText.MAX_BUFFER_SIZE, "1 b");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
 
@@ -311,7 +475,7 @@ public class TestReplaceText {
     public void testRoutesToSuccessIfTooLargeButRegexIsDotAsterisk() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, ".*");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.MAX_BUFFER_SIZE, "1 b");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
 
@@ -330,7 +494,7 @@ public class TestReplaceText {
     public void testProblematicCase1() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, ".*");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${filename}\t${now():format(\"yyyy/MM/dd'T'HHmmss'Z'\")}\t${fileSize}\n");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -351,7 +515,7 @@ public class TestReplaceText {
     public void testGetExistingContent() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, "(?s)(^.*)");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*)");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "attribute header\n\n${filename}\n\ndata header\n\n$1\n\nfooter");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -371,7 +535,7 @@ public class TestReplaceText {
     public void testReplaceWithinCurlyBraces() throws IOException {
         final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
         runner.setValidateExpressionUsage(false);
-        runner.setProperty(ReplaceText.REGEX, ".+");
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".+");
         runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "{ ${filename} }");
 
         final Map<String, String> attributes = new HashMap<>();
@@ -422,4 +586,421 @@ public class TestReplaceText {
         final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
         out.assertContentEquals(defaultValue);
     }
+
+    /* Line by Line */
+
+    @Test
+    public void testSimpleLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "odo");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "ood");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/food.txt")));
+    }
+
+    @Test
+    public void testPrependSimpleLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.PREPEND);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST ");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt")));
+    }
+
+    @Test
+    public void testAppendSimpleLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, " TEST");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt")));
+    }
+
+    @Test
+    public void testAppendEndlineCR() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+
+        runner.enqueue("Hello \rWorld \r".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Hello TEST\rWorld TEST\r".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testAppendEndlineCRLF() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "TEST");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.APPEND);
+
+        runner.enqueue("Hello \r\nWorld \r\n".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Hello TEST\r\nWorld TEST\r\n".getBytes("UTF-8"));
+    }
+
+    @Test
+    public void testSimpleLiteral() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "odo");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "ood");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/food.txt")));
+    }
+
+    @Test
+    public void testLiteralBackReferenceLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "jo");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt")));
+    }
+
+    @Test
+    public void testLiteralLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".ell.");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "test");
+        runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE);
+
+        runner.enqueue("H.ell.o, World! .ell. \n .ell. .ell.".getBytes());
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("Htesto, World! test \n test test");
+    }
+
+    @Test
+    public void testBackReferenceLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(DODO)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[$1]");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/[DODO].txt")));
+    }
+
+    @Test
+    public void testReplacementWithExpressionLanguageIsEscapedLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(jo)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "[${abc}]");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("abc", "$1");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt")));
+    }
+
+    @Test
+    public void testRegexWithExpressionLanguageLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("replaceKey", "Riley");
+        attributes.put("replaceValue", "Spider");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Spider.txt")));
+    }
+
+    @Test
+    public void testRegexWithExpressionLanguageIsEscapedLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "${replaceKey}");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${replaceValue}");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("replaceKey", "R.*y");
+        attributes.put("replaceValue", "Spider");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+    }
+
+    @Test
+    public void testBackReferenceWithTooLargeOfIndexIsEscapedLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(lu)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1$2");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("replaceKey", "R.*y");
+        attributes.put("replaceValue", "Spiderman");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt")));
+    }
+
+    @Test
+    public void testBackReferenceWithInvalidReferenceIsEscapedLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(ew)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$d");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("replaceKey", "H.*o");
+        attributes.put("replaceValue", "Good-bye");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt")));
+    }
+
+    @Test
+    public void testEscapingDollarSignLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(DO)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\\$1");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("replaceKey", "H.*o");
+        attributes.put("replaceValue", "Good-bye");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/$1$1.txt")));
+    }
+
+    @Test
+    public void testReplaceWithEmptyStringLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(jo)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/cu_Po.txt")));
+    }
+
+    @Test
+    public void testWithNoMatchLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "Z");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "Morning");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+    }
+
+    @Test
+    public void testWithMultipleMatchesLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "l");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "R");
+
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt")));
+    }
+
+    @Test
+    public void testAttributeToContentLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("abc", "Good");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals(translateNewLines(new File("src/test/resources/TestReplaceTextLineByLine/Good.txt")));
+    }
+
+    @Test
+    public void testAttributeToContentWindows() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${abc}");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("abc", "Good");
+        runner.enqueue("<<<HEADER>>>\r\n<<BODY>>\r\n<<<FOOTER>>>\r".getBytes(), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        out.assertContentEquals("GoodGoodGood");
+    }
+
+    @Test
+    public void testProblematicCase1LineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, ".*");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${filename}\t${now():format(\"yyyy/MM/dd'T'HHmmss'Z'\")}\t${fileSize}\n");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("filename", "abc.txt");
+        runner.enqueue(translateNewLines(Paths.get("src/test/resources/TestReplaceTextLineByLine/testFile.txt")), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        final String outContent = translateNewLines(new String(out.toByteArray(), StandardCharsets.UTF_8));
+        Assert.assertTrue(outContent.startsWith("abc.txt\t"));
+        System.out.println(outContent);
+        Assert.assertTrue(outContent.endsWith("193\n") || outContent.endsWith("203\r\n"));
+    }
+
+    @Test
+    public void testGetExistingContentLineByLine() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
+        runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*)");
+        runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "attribute header\n\n${filename}\n\ndata header\n\n$1\n\nfooter\n");
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("filename", "abc.txt");
+        runner.enqueue("Hello\nWorld!".getBytes(), attributes);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
+        final String outContent = new String(out.toByteArray(), StandardCharsets.UTF_8);
+        System.out.println(outContent);
+        Assert.assertTrue(outContent.equals("attribute header\n\nabc.txt\n\ndata header\n\nHello\n\n\nfooter\n"
+                + "attribute header\n\nabc.txt\n\ndata header\n\nWorld!\n\nfooter\n"));
+
+    }
+
+    private byte[] translateNewLines(final File file) throws IOException {
+        return translateNewLines(file.toPath());
+    }
+
+    private byte[] translateNewLines(final Path path) throws IOException {
+        final byte[] data = Files.readAllBytes(path);
+        final String text = new String(data, StandardCharsets.UTF_8);
+        return translateNewLines(text).getBytes(StandardCharsets.UTF_8);
+    }
+
+    private String translateNewLines(final String text) {
+        final String lineSeparator = System.getProperty("line.separator");
+        final Pattern pattern = Pattern.compile("\n", Pattern.MULTILINE);
+        final String translated = pattern.matcher(text).replaceAll(lineSeparator);
+        return translated;
+    }
 }


[24/50] [abbrv] nifi git commit: Removing nifi-pcap-bundle/.gitignore

Posted by ma...@apache.org.
Removing nifi-pcap-bundle/.gitignore


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

Branch: refs/heads/NIFI-730
Commit: c4f0cb1c6c5ff64eeff4cf98008ff838e4717abe
Parents: dc4004d
Author: Bryan Bende <bb...@apache.org>
Authored: Thu Oct 29 12:06:25 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Thu Oct 29 12:06:25 2015 -0400

----------------------------------------------------------------------
 nifi-nar-bundles/nifi-pcap-bundle/nifi-pcap-processors/.gitignore | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/c4f0cb1c/nifi-nar-bundles/nifi-pcap-bundle/nifi-pcap-processors/.gitignore
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-pcap-bundle/nifi-pcap-processors/.gitignore b/nifi-nar-bundles/nifi-pcap-bundle/nifi-pcap-processors/.gitignore
deleted file mode 100644
index ae3c172..0000000
--- a/nifi-nar-bundles/nifi-pcap-bundle/nifi-pcap-processors/.gitignore
+++ /dev/null
@@ -1 +0,0 @@
-/bin/


[13/50] [abbrv] nifi git commit: NIFI-10: Added FETCH and DOWNLOAD Provenance Events; updated FlowController to use DOWNLOAD event instead of SEND whenever a user downloads/views content via Provenance Event

Posted by ma...@apache.org.
NIFI-10: Added FETCH and DOWNLOAD Provenance Events; updated FlowController to use DOWNLOAD event instead of SEND whenever a user downloads/views content via Provenance Event


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

Branch: refs/heads/NIFI-730
Commit: fc2aa2764cc9e85a19d3f3eec640873f43c60148
Parents: 51f5640
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 25 11:53:46 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 14:58:50 2015 -0400

----------------------------------------------------------------------
 .../nifi/provenance/ProvenanceEventType.java    |   5 +
 .../apache/nifi/controller/FlowController.java  | 130 +++++++++----------
 2 files changed, 69 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/fc2aa276/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
index 188e8fc..0d844b8 100644
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
+++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventType.java
@@ -47,6 +47,11 @@ public enum ProvenanceEventType {
     SEND,
 
     /**
+     * Indicates that the contents of a FlowFile were downloaded by a user or external entity.
+     */
+    DOWNLOAD,
+
+    /**
      * Indicates a provenance event for the conclusion of an object's life for
      * some reason other than object expiration
      */

http://git-wip-us.apache.org/repos/asf/nifi/blob/fc2aa276/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index d9c3f39..3f815b0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -216,7 +216,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     public static final String SCHEDULE_MINIMUM_NANOSECONDS = "flowcontroller.minimum.nanoseconds";
     public static final String GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.seconds";
     public static final long DEFAULT_GRACEFUL_SHUTDOWN_SECONDS = 10;
-    public static final int METRICS_RESERVOIR_SIZE = 288;   // 1 day worth of 5-minute captures
+    public static final int METRICS_RESERVOIR_SIZE = 288; // 1 day worth of 5-minute captures
 
     public static final String ROOT_GROUP_ID_ALIAS = "root";
     public static final String DEFAULT_ROOT_GROUP_NAME = "NiFi Flow";
@@ -245,7 +245,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private final UserService userService;
     private final EventDrivenWorkerQueue eventDrivenWorkerQueue;
     private final ComponentStatusRepository componentStatusRepository;
-    private final long systemStartTime = System.currentTimeMillis();    // time at which the node was started
+    private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started
     private final ConcurrentMap<String, ReportingTaskNode> reportingTasks = new ConcurrentHashMap<>();
 
     // The Heartbeat Bean is used to provide an Atomic Reference to data that is used in heartbeats that may
@@ -336,38 +336,38 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private final Lock readLock = rwLock.readLock();
     private final Lock writeLock = rwLock.writeLock();
 
-    private FlowFileSwapManager flowFileSwapManager;    // guarded by read/write lock
+    private FlowFileSwapManager flowFileSwapManager; // guarded by read/write lock
 
     private static final Logger LOG = LoggerFactory.getLogger(FlowController.class);
     private static final Logger heartbeatLogger = LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat");
 
     public static FlowController createStandaloneInstance(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor) {
+        final FlowFileEventRepository flowFileEventRepo,
+        final NiFiProperties properties,
+        final UserService userService,
+        final StringEncryptor encryptor) {
         return new FlowController(
-                flowFileEventRepo,
-                properties,
-                userService,
-                encryptor,
-                /* configuredForClustering */ false,
-                /* NodeProtocolSender */ null);
+            flowFileEventRepo,
+            properties,
+            userService,
+            encryptor,
+            /* configuredForClustering */ false,
+            /* NodeProtocolSender */ null);
     }
 
     public static FlowController createClusteredInstance(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor,
-            final NodeProtocolSender protocolSender) {
+        final FlowFileEventRepository flowFileEventRepo,
+        final NiFiProperties properties,
+        final UserService userService,
+        final StringEncryptor encryptor,
+        final NodeProtocolSender protocolSender) {
         final FlowController flowController = new FlowController(
-                flowFileEventRepo,
-                properties,
-                userService,
-                encryptor,
-                /* configuredForClustering */ true,
-                /* NodeProtocolSender */ protocolSender);
+            flowFileEventRepo,
+            properties,
+            userService,
+            encryptor,
+            /* configuredForClustering */ true,
+            /* NodeProtocolSender */ protocolSender);
 
         flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.isSiteToSiteSecure());
 
@@ -375,12 +375,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     private FlowController(
-            final FlowFileEventRepository flowFileEventRepo,
-            final NiFiProperties properties,
-            final UserService userService,
-            final StringEncryptor encryptor,
-            final boolean configuredForClustering,
-            final NodeProtocolSender protocolSender) {
+        final FlowFileEventRepository flowFileEventRepo,
+        final NiFiProperties properties,
+        final UserService userService,
+        final StringEncryptor encryptor,
+        final boolean configuredForClustering,
+        final NodeProtocolSender protocolSender) {
 
         maxTimerDrivenThreads = new AtomicInteger(10);
         maxEventDrivenThreads = new AtomicInteger(5);
@@ -416,7 +416,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceEventRepository);
         processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
-                eventDrivenEngineRef.get(), this, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
+            eventDrivenEngineRef.get(), this, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
 
         final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
         final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
@@ -468,7 +468,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             externalSiteListener = null;
         } else if (isSiteToSiteSecure && sslContext == null) {
             LOG.error("Unable to create Secure Site-to-Site Listener because not all required Keystore/Truststore "
-                    + "Properties are set. Site-to-Site functionality will be disabled until this problem is has been fixed.");
+                + "Properties are set. Site-to-Site functionality will be disabled until this problem is has been fixed.");
             externalSiteListener = null;
         } else {
             // Register the SocketFlowFileServerProtocol as the appropriate resource for site-to-site Server Protocol
@@ -501,7 +501,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION);
         if (implementationClassName == null) {
             throw new RuntimeException("Cannot create FlowFile Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION);
+                + NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION);
         }
 
         try {
@@ -612,7 +612,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                             startConnectable(connectable);
                         }
                     } catch (final Throwable t) {
-                        LOG.error("Unable to start {} due to {}", new Object[]{connectable, t.toString()});
+                        LOG.error("Unable to start {} due to {}", new Object[] {connectable, t.toString()});
                         if (LOG.isDebugEnabled()) {
                             LOG.error("", t);
                         }
@@ -627,7 +627,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                         remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort);
                         startedTransmitting++;
                     } catch (final Throwable t) {
-                        LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t});
+                        LOG.error("Unable to start transmitting with {} due to {}", new Object[] {remoteGroupPort, t});
                     }
                 }
 
@@ -642,7 +642,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                             startConnectable(connectable);
                         }
                     } catch (final Throwable t) {
-                        LOG.error("Unable to start {} due to {}", new Object[]{connectable, t});
+                        LOG.error("Unable to start {} due to {}", new Object[] {connectable, t});
                     }
                 }
 
@@ -658,7 +658,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
         if (implementationClassName == null) {
             throw new RuntimeException("Cannot create Provenance Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION);
+                + NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION);
         }
 
         try {
@@ -676,7 +676,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final String implementationClassName = properties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, DEFAULT_PROVENANCE_REPO_IMPLEMENTATION);
         if (implementationClassName == null) {
             throw new RuntimeException("Cannot create Provenance Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
+                + NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
         }
 
         try {
@@ -690,7 +690,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
         if (implementationClassName == null) {
             throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
-                    + NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
+                + NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
         }
 
         try {
@@ -910,7 +910,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         name = requireNonNull(name).intern();
         verifyPortIdDoesNotExist(id);
         return new StandardRootGroupPort(id, name, null, TransferDirection.RECEIVE, ConnectableType.INPUT_PORT,
-                userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
+            userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
     }
 
     /**
@@ -927,7 +927,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         name = requireNonNull(name).intern();
         verifyPortIdDoesNotExist(id);
         return new StandardRootGroupPort(id, name, null, TransferDirection.SEND, ConnectableType.OUTPUT_PORT,
-                userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
+            userService, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
     }
 
     /**
@@ -1083,14 +1083,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             try {
                 flowFileRepository.close();
             } catch (final Throwable t) {
-                LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[]{t});
+                LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[] {t});
             }
 
             if (this.timerDrivenEngineRef.get().isTerminated() && eventDrivenEngineRef.get().isTerminated()) {
                 LOG.info("Controller has been terminated successfully.");
             } else {
                 LOG.warn("Controller hasn't terminated properly.  There exists an uninterruptable thread that "
-                        + "will take an indeterminate amount of time to stop.  Might need to kill the program manually.");
+                    + "will take an indeterminate amount of time to stop.  Might need to kill the program manually.");
             }
 
             if (externalSiteListener != null) {
@@ -1153,7 +1153,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      * @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used
      */
     public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow)
-            throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
+        throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
         writeLock.lock();
         try {
             LOG.debug("Synchronizing controller with proposed flow");
@@ -1199,7 +1199,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      *
      * @param maxThreadCount
      *
-     * This method must be called while holding the write lock!
+     *            This method must be called while holding the write lock!
      */
     private void setMaxThreadCount(final int maxThreadCount, final FlowEngine engine, final AtomicInteger maxThreads) {
         if (maxThreadCount < 1) {
@@ -1267,7 +1267,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      * @throws ProcessorInstantiationException
      *
      * @throws IllegalStateException if no process group can be found with the ID of DTO or with the ID of the DTO's parentGroupId, if the template ID specified is invalid, or if the DTO's Parent
-     * Group ID changes but the parent group has incoming or outgoing connections
+     *             Group ID changes but the parent group has incoming or outgoing connections
      *
      * @throws NullPointerException if the DTO or its ID is null
      */
@@ -1371,7 +1371,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      *
      * @throws NullPointerException if either argument is null
      * @throws IllegalStateException if the snippet is not valid because a component in the snippet has an ID that is not unique to this flow, or because it shares an Input Port or Output Port at the
-     * root level whose name already exists in the given ProcessGroup, or because the Template contains a Processor or a Prioritizer whose class is not valid within this instance of NiFi.
+     *             root level whose name already exists in the given ProcessGroup, or because the Template contains a Processor or a Prioritizer whose class is not valid within this instance of NiFi.
      * @throws ProcessorInstantiationException if unable to instantiate a processor
      */
     public void instantiateSnippet(final ProcessGroup group, final FlowSnippetDTO dto) throws ProcessorInstantiationException {
@@ -2542,7 +2542,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         if (firstTimeAdded) {
             final ComponentLog componentLog = new SimpleProcessLogger(id, taskNode.getReportingTask());
             final ReportingInitializationContext config = new StandardReportingInitializationContext(id, taskNode.getName(),
-                    SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this);
+                SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this);
 
             try {
                 task.initialize(config);
@@ -2888,7 +2888,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         readLock.lock();
         try {
             return heartbeatGeneratorFuture != null && !heartbeatGeneratorFuture.isCancelled()
-                    && heartbeatSenderFuture != null && !heartbeatSenderFuture.isCancelled();
+                && heartbeatSenderFuture != null && !heartbeatSenderFuture.isCancelled();
         } finally {
             readLock.unlock();
         }
@@ -2948,7 +2948,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
     /**
      * @return the DN of the Cluster Manager that we are currently connected to, if available. This will return null if the instance is not clustered or if the instance is clustered but the NCM's DN
-     * is not available - for instance, if cluster communications are not secure
+     *         is not available - for instance, if cluster communications are not secure
      */
     public String getClusterManagerDN() {
         readLock.lock();
@@ -3101,10 +3101,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             @Override
             public boolean isContentSame() {
                 return areEqual(event.getPreviousContentClaimContainer(), event.getContentClaimContainer())
-                        && areEqual(event.getPreviousContentClaimSection(), event.getContentClaimSection())
-                        && areEqual(event.getPreviousContentClaimIdentifier(), event.getContentClaimIdentifier())
-                        && areEqual(event.getPreviousContentClaimOffset(), event.getContentClaimOffset())
-                        && areEqual(event.getPreviousFileSize(), event.getFileSize());
+                    && areEqual(event.getPreviousContentClaimSection(), event.getContentClaimSection())
+                    && areEqual(event.getPreviousContentClaimIdentifier(), event.getContentClaimIdentifier())
+                    && areEqual(event.getPreviousContentClaimOffset(), event.getContentClaimOffset())
+                    && areEqual(event.getPreviousFileSize(), event.getFileSize());
             }
 
             @Override
@@ -3180,7 +3180,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // Register a Provenance Event to indicate that we replayed the data.
         final ProvenanceEventRecord sendEvent = new StandardProvenanceEventRecord.Builder()
-            .setEventType(ProvenanceEventType.SEND)
+            .setEventType(ProvenanceEventType.DOWNLOAD)
             .setFlowFileUUID(provEvent.getFlowFileUuid())
             .setAttributes(provEvent.getAttributes(), Collections.<String, String> emptyMap())
             .setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(), offset, size)
@@ -3297,7 +3297,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // Create the ContentClaim
         final ResourceClaim resourceClaim = contentClaimManager.newResourceClaim(event.getPreviousContentClaimContainer(),
-                event.getPreviousContentClaimSection(), event.getPreviousContentClaimIdentifier(), false);
+            event.getPreviousContentClaimSection(), event.getPreviousContentClaimIdentifier(), false);
 
         // Increment Claimant Count, since we will now be referencing the Content Claim
         contentClaimManager.incrementClaimantCount(resourceClaim);
@@ -3367,7 +3367,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         // Update the FlowFile Repository to indicate that we have added the FlowFile to the flow
         final StandardRepositoryRecord record = new StandardRepositoryRecord(queue, flowFileRecord);
         record.setDestination(queue);
-        flowFileRepository.updateRepository(Collections.<RepositoryRecord>singleton(record));
+        flowFileRepository.updateRepository(Collections.<RepositoryRecord> singleton(record));
 
         // Enqueue the data
         queue.put(flowFileRecord);
@@ -3434,11 +3434,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 protocolSender.sendBulletins(message);
                 if (LOG.isDebugEnabled()) {
                     LOG.debug(
-                            String.format(
-                                    "Sending bulletins to cluster manager at %s",
-                                    dateFormatter.format(new Date())
-                            )
-                    );
+                        String.format(
+                            "Sending bulletins to cluster manager at %s",
+                            dateFormatter.format(new Date())));
                 }
 
             } catch (final UnknownServiceAddressException usae) {
@@ -3496,7 +3494,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                         escapedBulletin = BulletinFactory.createBulletin(bulletin.getCategory(), bulletin.getLevel(), escapedBulletinMessage);
                     } else {
                         escapedBulletin = BulletinFactory.createBulletin(bulletin.getGroupId(), bulletin.getSourceId(), bulletin.getSourceType(),
-                                bulletin.getSourceName(), bulletin.getCategory(), bulletin.getLevel(), escapedBulletinMessage);
+                            bulletin.getSourceName(), bulletin.getCategory(), bulletin.getLevel(), escapedBulletinMessage);
                     }
                 } else {
                     escapedBulletin = bulletin;
@@ -3554,9 +3552,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 final long sendMillis = TimeUnit.NANOSECONDS.toMillis(sendNanos);
 
                 heartbeatLogger.info("Heartbeat created at {} and sent at {}; send took {} millis",
-                        dateFormatter.format(new Date(message.getHeartbeat().getCreatedTimestamp())),
-                        dateFormatter.format(new Date()),
-                        sendMillis);
+                    dateFormatter.format(new Date(message.getHeartbeat().getCreatedTimestamp())),
+                    dateFormatter.format(new Date()),
+                    sendMillis);
             } catch (final UnknownServiceAddressException usae) {
                 if (heartbeatLogger.isDebugEnabled()) {
                     heartbeatLogger.debug(usae.getMessage());


[04/50] [abbrv] nifi git commit: NIFI-673: Added sftp.listing.user attribute to FlowFiles created by ListSFTP; ensure that FetchSFTP indicates that the username supports Expression Language

Posted by ma...@apache.org.
NIFI-673: Added sftp.listing.user attribute to FlowFiles created by ListSFTP; ensure that FetchSFTP indicates that the username supports Expression Language


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

Branch: refs/heads/NIFI-730
Commit: 4e38288062f33ecafdfd3b4bc453bdffe54e752e
Parents: b0322d9
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Oct 23 16:13:52 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:13:02 2015 -0400

----------------------------------------------------------------------
 .../apache/nifi/processors/standard/FetchFileTransfer.java   | 7 +++++++
 .../java/org/apache/nifi/processors/standard/FetchSFTP.java  | 2 +-
 .../apache/nifi/processors/standard/ListFileTransfer.java    | 8 ++++++++
 .../java/org/apache/nifi/processors/standard/ListSFTP.java   | 3 ++-
 4 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/4e382880/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
index ab0be78..a405afb 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchFileTransfer.java
@@ -77,6 +77,13 @@ public abstract class FetchFileTransfer extends AbstractProcessor {
         .expressionLanguageSupported(true)
         .required(true)
         .build();
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+        .name("Username")
+        .description("Username")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
     public static final PropertyDescriptor REMOTE_FILENAME = new PropertyDescriptor.Builder()
         .name("Remote File")
         .description("The fully qualified filename on the remote system")

http://git-wip-us.apache.org/repos/asf/nifi/blob/4e382880/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
index ad81c83..b95d864 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FetchSFTP.java
@@ -53,7 +53,7 @@ public class FetchSFTP extends FetchFileTransfer {
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(HOSTNAME);
         properties.add(port);
-        properties.add(SFTPTransfer.USERNAME);
+        properties.add(USERNAME);
         properties.add(SFTPTransfer.PASSWORD);
         properties.add(SFTPTransfer.PRIVATE_KEY_PATH);
         properties.add(SFTPTransfer.PRIVATE_KEY_PASSPHRASE);

http://git-wip-us.apache.org/repos/asf/nifi/blob/4e382880/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
index 1176fd0..b6c8c28 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListFileTransfer.java
@@ -45,6 +45,13 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
         .expressionLanguageSupported(true)
         .required(true)
         .build();
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+        .name("Username")
+        .description("Username")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(true)
+        .required(true)
+        .build();
     public static final PropertyDescriptor REMOTE_PATH = new PropertyDescriptor.Builder()
         .name("Remote Path")
         .description("The path on the remote system from which to pull or push files")
@@ -64,6 +71,7 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
         attributes.put("file.group", fileInfo.getGroup());
         attributes.put("file.permissions", fileInfo.getPermissions());
         attributes.put(CoreAttributes.FILENAME.key(), fileInfo.getFileName());
+        attributes.put(getProtocolName() + ".listing.user", context.getProperty(USERNAME).evaluateAttributeExpressions().getValue());
 
         final String fullPath = fileInfo.getFullPathFileName();
         if (fullPath != null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/4e382880/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
index 925e5f8..7226263 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListSFTP.java
@@ -39,6 +39,7 @@ import org.apache.nifi.processors.standard.util.SFTPTransfer;
 @WritesAttributes({
     @WritesAttribute(attribute = "sftp.remote.host", description = "The hostname of the SFTP Server"),
     @WritesAttribute(attribute = "sftp.remote.port", description = "The port that was connected to on the SFTP Server"),
+    @WritesAttribute(attribute = "sftp.listing.user", description = "The username of the user that performed the SFTP Listing"),
     @WritesAttribute(attribute = "file.owner", description = "The numeric owner id of the source file"),
     @WritesAttribute(attribute = "file.group", description = "The numeric group id of the source file"),
     @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the source file"),
@@ -54,7 +55,7 @@ public class ListSFTP extends ListFileTransfer {
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(HOSTNAME);
         properties.add(port);
-        properties.add(SFTPTransfer.USERNAME);
+        properties.add(USERNAME);
         properties.add(SFTPTransfer.PASSWORD);
         properties.add(SFTPTransfer.PRIVATE_KEY_PATH);
         properties.add(SFTPTransfer.PRIVATE_KEY_PASSPHRASE);


[31/50] [abbrv] nifi git commit: NIFI-1079 Added Destination Property to control if JSON goes to Attribute or Content of FlowFile. Added Include Core Attributes Property to control if FlowFile CoreAttributes are included in the JSON output or not. Added

Posted by ma...@apache.org.
NIFI-1079 Added Destination Property to control if JSON goes to Attribute or
Content of FlowFile. Added Include Core Attributes Property to control
if FlowFile CoreAttributes are included in the JSON output or not.
Added Null value for Empty String flag to control if empty values in
the JSON are empty string or true NULL values. Added more tests and
minor text refactoring per Github comments

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/NIFI-730
Commit: 217b1049cf73c8ecef7dd76a954f07a7c7224cc6
Parents: 19b7a4c
Author: Jeremy Dyer <jd...@gmail.com>
Authored: Wed Oct 28 20:53:46 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Oct 30 14:30:25 2015 -0400

----------------------------------------------------------------------
 .../processors/standard/AttributesToJSON.java   | 151 +++++++++++++++----
 .../standard/TestAttributesToJSON.java          | 116 ++++++++++++--
 2 files changed, 224 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/217b1049/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
index 7098b6e..950b8d3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
@@ -8,16 +8,22 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.*;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.stream.io.BufferedOutputStream;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.util.*;
 
 @EventDriven
 @SideEffectFree
 @SupportsBatching
-@Tags({"JSON", "attributes"})
+@Tags({"json", "attributes"})
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
 @CapabilityDescription("Evaluates the attributes from a FlowFile and generates a JSON string with the attribute key/value pair. " +
         "The resulting JSON string is placed in the FlowFile as a new Attribute with the name 'JSONAttributes'. By default all" +
@@ -28,16 +34,48 @@ public class AttributesToJSON extends AbstractProcessor {
 
     public static final String JSON_ATTRIBUTE_NAME = "JSONAttribute";
     private static final String AT_LIST_SEPARATOR = ",";
-    private static final String DEFAULT_VALUE_IF_ATT_NOT_PRESENT = "";
+
+    public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
+    public static final String DESTINATION_CONTENT = "flowfile-content";
+
 
     public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder()
             .name("Attributes List")
-            .description("Comma separated list of attributes to be included in the '" + JSON_ATTRIBUTE_NAME +"' attribute. This list of attributes is case sensitive. If a specified attribute is not found" +
+            .description("Comma separated list of attributes to be included in the '" + JSON_ATTRIBUTE_NAME +"' " +
+                    "attribute. This list of attributes is case sensitive. If a specified attribute is not found" +
                     "in the flowfile an empty string will be output for that attritbute in the resulting JSON")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
+    public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
+            .name("Destination")
+            .description("Control if JSON value is written as a new flowfile attribute '" + JSON_ATTRIBUTE_NAME + "' " +
+                    "or written in the flowfile content. " +
+                    "Writing to flowfile content will overwrite any existing flowfile content.")
+            .required(true)
+            .allowableValues(DESTINATION_ATTRIBUTE, DESTINATION_CONTENT)
+            .defaultValue(DESTINATION_ATTRIBUTE)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_CORE_ATTRIBUTES = new PropertyDescriptor.Builder()
+            .name("Include Core Attributes")
+            .description("Determines if the FlowFile org.apache.nifi.flowfile.attributes.CoreAttributes which are " +
+                    "contained in every FlowFile should be included in the final JSON value generated.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor NULL_VALUE_FOR_EMPTY_STRING = new PropertyDescriptor.Builder()
+            .name("Null Value")
+            .description("If an Attribute is value is empty or not present this property determines if an empty string" +
+                    "or true NULL value is present in the resulting JSON output")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
             .description("'" + JSON_ATTRIBUTE_NAME + "' attribute has been successfully added to the flowfile").build();
@@ -52,6 +90,9 @@ public class AttributesToJSON extends AbstractProcessor {
     protected void init(final ProcessorInitializationContext context) {
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(ATTRIBUTES_LIST);
+        properties.add(DESTINATION);
+        properties.add(INCLUDE_CORE_ATTRIBUTES);
+        properties.add(NULL_VALUE_FOR_EMPTY_STRING);
         this.properties = Collections.unmodifiableList(properties);
 
         final Set<Relationship> relationships = new HashSet<>();
@@ -70,48 +111,100 @@ public class AttributesToJSON extends AbstractProcessor {
         return relationships;
     }
 
-    @Override
-    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-        final FlowFile original = session.get();
-        if (original == null) {
-            return;
-        }
 
-        final String atList = context.getProperty(ATTRIBUTES_LIST).getValue();
-        Map<String, String> atsToWrite = null;
+    /**
+     * Builds the Map of attributes that should be included in the JSON that is emitted from this process.
+     *
+     * @return
+     *  Map<String, String> of values that are feed to a Jackson ObjectMapper
+     */
+    protected Map<String, String> buildAttributesMapForFlowFile(FlowFile ff, String atrList,
+                                                                boolean includeCoreAttributes,
+                                                                boolean nullValForEmptyString) {
+
+        Map<String, String> atsToWrite = new HashMap<>();
 
         //If list of attributes specified get only those attributes. Otherwise write them all
-        if (atList != null && !StringUtils.isEmpty(atList)) {
-            atsToWrite = new HashMap<>();
-            String[] ats = StringUtils.split(atList, AT_LIST_SEPARATOR);
+        if (StringUtils.isNotBlank(atrList)) {
+            String[] ats = StringUtils.split(atrList, AT_LIST_SEPARATOR);
             if (ats != null) {
                 for (String str : ats) {
                     String cleanStr = str.trim();
-                    String val = original.getAttribute(cleanStr);
+                    String val = ff.getAttribute(cleanStr);
                     if (val != null) {
                         atsToWrite.put(cleanStr, val);
                     } else {
-                        atsToWrite.put(cleanStr, DEFAULT_VALUE_IF_ATT_NOT_PRESENT);
+                        if (nullValForEmptyString) {
+                            atsToWrite.put(cleanStr, null);
+                        } else {
+                            atsToWrite.put(cleanStr, "");
+                        }
                     }
                 }
             }
         } else {
-            atsToWrite = original.getAttributes();
+            atsToWrite.putAll(ff.getAttributes());
         }
 
-        if (atsToWrite != null) {
-            if (atsToWrite.size() == 0) {
-                getLogger().debug("Flowfile contains no attributes to convert to JSON");
-            } else {
-                try {
-                    FlowFile updated = session.putAttribute(original, JSON_ATTRIBUTE_NAME, objectMapper.writeValueAsString(atsToWrite));
-                    session.transfer(updated, REL_SUCCESS);
-                } catch (JsonProcessingException e) {
-                    getLogger().error(e.getMessage());
-                    session.transfer(original, REL_FAILURE);
-                }
-            }
+        if (!includeCoreAttributes) {
+            atsToWrite = removeCoreAttributes(atsToWrite);
+        }
+
+        return atsToWrite;
+    }
+
+    /**
+     * Remove all of the CoreAttributes from the Attributes that will be written to the Flowfile.
+     *
+     * @param atsToWrite
+     *  List of Attributes that have already been generated including the CoreAttributes
+     *
+     * @return
+     *  Difference of all attributes minus the CoreAttributes
+     */
+    protected Map<String, String> removeCoreAttributes(Map<String, String> atsToWrite) {
+        for (CoreAttributes c : CoreAttributes.values()) {
+            atsToWrite.remove(c.key());
+        }
+        return atsToWrite;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        final FlowFile original = session.get();
+        if (original == null) {
+            return;
         }
 
+        final Map<String, String> atrList = buildAttributesMapForFlowFile(original,
+                context.getProperty(ATTRIBUTES_LIST).getValue(),
+                context.getProperty(INCLUDE_CORE_ATTRIBUTES).asBoolean(),
+                context.getProperty(NULL_VALUE_FOR_EMPTY_STRING).asBoolean());
+
+        try {
+
+            switch (context.getProperty(DESTINATION).getValue()) {
+                case DESTINATION_ATTRIBUTE:
+                    FlowFile atFlowfile = session.putAttribute(original, JSON_ATTRIBUTE_NAME,
+                            objectMapper.writeValueAsString(atrList));
+                    session.transfer(atFlowfile, REL_SUCCESS);
+                    break;
+                case DESTINATION_CONTENT:
+                    FlowFile conFlowfile = session.write(original, new StreamCallback() {
+                        @Override
+                        public void process(InputStream in, OutputStream out) throws IOException {
+                            try (OutputStream outputStream = new BufferedOutputStream(out)) {
+                                outputStream.write(objectMapper.writeValueAsBytes(atrList));
+                            }
+                        }
+                    });
+                    session.transfer(conFlowfile, REL_SUCCESS);
+                    break;
+            }
+
+        } catch (JsonProcessingException e) {
+            getLogger().error(e.getMessage());
+            session.transfer(original, REL_FAILURE);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/217b1049/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
index a057d15..1624c4b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
@@ -6,28 +6,17 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 
 public class TestAttributesToJSON {
 
-    private static Logger LOGGER;
-
-    static {
-        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
-        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.AttributesToJSON", "debug");
-        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestAttributesToJSON", "debug");
-        LOGGER = LoggerFactory.getLogger(TestAttributesToJSON.class);
-    }
-
     private static final String TEST_ATTRIBUTE_KEY = "TestAttribute";
     private static final String TEST_ATTRIBUTE_VALUE = "TestValue";
 
@@ -45,9 +34,84 @@ public class TestAttributesToJSON {
         testRunner.run();
     }
 
+    @Test(expected = AssertionError.class)
+    public void testInvalidIncludeCoreAttributesProperty() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, "val1,val2");
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
+        testRunner.setProperty(AttributesToJSON.INCLUDE_CORE_ATTRIBUTES, "maybe");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+    }
+
+    @Test
+    public void testNullValueForEmptyAttribute() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
+        final String NON_PRESENT_ATTRIBUTE_KEY = "NonExistingAttributeKey";
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY);
+        testRunner.setProperty(AttributesToJSON.NULL_VALUE_FOR_EMPTY_STRING, "true");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        //Make sure that the value is a true JSON null for the non existing attribute
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+
+        assertNull(val.get(NON_PRESENT_ATTRIBUTE_KEY));
+    }
+
+    @Test
+    public void testEmptyStringValueForEmptyAttribute() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
+        final String NON_PRESENT_ATTRIBUTE_KEY = "NonExistingAttributeKey";
+        testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY);
+        testRunner.setProperty(AttributesToJSON.NULL_VALUE_FOR_EMPTY_STRING, "false");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+
+        //Make sure that the value is a true JSON null for the non existing attribute
+        String json = testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS)
+                .get(0).getAttribute(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+
+        ObjectMapper mapper = new ObjectMapper();
+        Map<String, String> val = mapper.readValue(json, HashMap.class);
+
+        assertEquals(val.get(NON_PRESENT_ATTRIBUTE_KEY), "");
+    }
+
     @Test
     public void testInvalidJSONValueInAttribute() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
 
         ProcessSession session = testRunner.getProcessSessionFactory().createSession();
         FlowFile ff = session.create();
@@ -67,8 +131,9 @@ public class TestAttributesToJSON {
 
 
     @Test
-    public void testAttribuets_emptyListUserSpecifiedAttributes() throws Exception {
+    public void testAttributes_emptyListUserSpecifiedAttributes() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
 
         ProcessSession session = testRunner.getProcessSessionFactory().createSession();
         FlowFile ff = session.create();
@@ -93,9 +158,30 @@ public class TestAttributesToJSON {
 
 
     @Test
+    public void testContent_emptyListUserSpecifiedAttributes() throws Exception {
+        final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_CONTENT);
+        testRunner.setProperty(AttributesToJSON.INCLUDE_CORE_ATTRIBUTES, "false");
+
+        ProcessSession session = testRunner.getProcessSessionFactory().createSession();
+        FlowFile ff = session.create();
+
+        testRunner.enqueue(ff);
+        testRunner.run();
+
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
+                assertAttributeNotExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
+        testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).assertContentEquals("{}");
+    }
+
+
+    @Test
     public void testAttribute_singleUserDefinedAttribute() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
         testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, TEST_ATTRIBUTE_KEY);
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
 
         ProcessSession session = testRunner.getProcessSessionFactory().createSession();
         FlowFile ff = session.create();
@@ -123,6 +209,7 @@ public class TestAttributesToJSON {
     public void testAttribute_singleUserDefinedAttributeWithWhiteSpace() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
         testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, " " + TEST_ATTRIBUTE_KEY + " ");
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
 
         ProcessSession session = testRunner.getProcessSessionFactory().createSession();
         FlowFile ff = session.create();
@@ -150,6 +237,7 @@ public class TestAttributesToJSON {
     public void testAttribute_singleNonExistingUserDefinedAttribute() throws Exception {
         final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToJSON());
         testRunner.setProperty(AttributesToJSON.ATTRIBUTES_LIST, "NonExistingAttribute");
+        testRunner.setProperty(AttributesToJSON.DESTINATION, AttributesToJSON.DESTINATION_ATTRIBUTE);
 
         ProcessSession session = testRunner.getProcessSessionFactory().createSession();
         FlowFile ff = session.create();


[32/50] [abbrv] nifi git commit: NIFI-1079 Check style and documentation based updates recommended by user bbende on Github pull request

Posted by ma...@apache.org.
NIFI-1079 Check style and documentation based updates recommended by user bbende on Github pull request

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/NIFI-730
Commit: aef0d8fe9b994ce333d14083d7f39edc77622d72
Parents: 217b104
Author: Jeremy Dyer <jd...@gmail.com>
Authored: Fri Oct 30 12:37:11 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Oct 30 14:30:42 2015 -0400

----------------------------------------------------------------------
 .../processors/standard/AttributesToJSON.java   | 74 ++++++++++++++------
 .../standard/TestAttributesToJSON.java          | 49 ++++++++-----
 2 files changed, 86 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/aef0d8fe/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
index 950b8d3..89bb0b6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToJSON.java
@@ -1,15 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.nifi.processors.standard;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.annotation.behavior.*;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.*;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
@@ -18,32 +43,38 @@ import org.apache.nifi.stream.io.BufferedOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Collections;
 
 @EventDriven
 @SideEffectFree
 @SupportsBatching
-@Tags({"json", "attributes"})
+@Tags({"json", "attributes", "flowfile"})
 @InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-@CapabilityDescription("Evaluates the attributes from a FlowFile and generates a JSON string with the attribute key/value pair. " +
-        "The resulting JSON string is placed in the FlowFile as a new Attribute with the name 'JSONAttributes'. By default all" +
-        "Attributes in the FlowFile are placed in the resulting JSON string. If only certain Attributes are desired you may" +
-        "specify a list of FlowFile Attributes that you want in the resulting JSON string")
-@WritesAttribute(attribute = "JSONAttributes", description = "JSON string of all the pre-existing attributes in the flowfile")
+@CapabilityDescription("Generates a JSON representation of the input FlowFile Attributes. The resulting JSON " +
+        "can be written to either a new Attribute 'JSONAttributes' or written to the FlowFile as content.")
+@WritesAttribute(attribute = "JSONAttributes", description = "JSON representation of Attributes")
 public class AttributesToJSON extends AbstractProcessor {
 
-    public static final String JSON_ATTRIBUTE_NAME = "JSONAttribute";
+    public static final String JSON_ATTRIBUTE_NAME = "JSONAttributes";
     private static final String AT_LIST_SEPARATOR = ",";
 
     public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
     public static final String DESTINATION_CONTENT = "flowfile-content";
+    private final String APPLICATION_JSON = "application/json";
 
 
     public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder()
             .name("Attributes List")
-            .description("Comma separated list of attributes to be included in the '" + JSON_ATTRIBUTE_NAME +"' " +
-                    "attribute. This list of attributes is case sensitive. If a specified attribute is not found" +
-                    "in the flowfile an empty string will be output for that attritbute in the resulting JSON")
+            .description("Comma separated list of attributes to be included in the resulting JSON. If this value " +
+                    "is left empty then all existing Attributes will be included. This list of attributes is " +
+                    "case sensitive. If an attribute specified in the list is not found it will be be emitted " +
+                    "to the resulting JSON with an empty string or NULL value.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
@@ -51,8 +82,8 @@ public class AttributesToJSON extends AbstractProcessor {
     public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
             .name("Destination")
             .description("Control if JSON value is written as a new flowfile attribute '" + JSON_ATTRIBUTE_NAME + "' " +
-                    "or written in the flowfile content. " +
-                    "Writing to flowfile content will overwrite any existing flowfile content.")
+                    "or written in the flowfile content. Writing to flowfile content will overwrite any " +
+                    "existing flowfile content.")
             .required(true)
             .allowableValues(DESTINATION_ATTRIBUTE, DESTINATION_CONTENT)
             .defaultValue(DESTINATION_ATTRIBUTE)
@@ -68,9 +99,9 @@ public class AttributesToJSON extends AbstractProcessor {
             .build();
 
     public static final PropertyDescriptor NULL_VALUE_FOR_EMPTY_STRING = new PropertyDescriptor.Builder()
-            .name("Null Value")
-            .description("If an Attribute is value is empty or not present this property determines if an empty string" +
-                    "or true NULL value is present in the resulting JSON output")
+            .name(("If true a non existing or empty attribute will be NULL in the resulting JSON. If false an empty " +
+                    "string will be placed in the JSON"))
+            .description("")
             .required(true)
             .allowableValues("true", "false")
             .defaultValue("false")
@@ -78,9 +109,9 @@ public class AttributesToJSON extends AbstractProcessor {
 
 
     public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
-            .description("'" + JSON_ATTRIBUTE_NAME + "' attribute has been successfully added to the flowfile").build();
+            .description("Successfully converted attributes to JSON").build();
     public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("Failed to add '" + JSON_ATTRIBUTE_NAME + "' attribute to the flowfile").build();
+            .description("Failed to convert attributes to JSON").build();
 
     private List<PropertyDescriptor> properties;
     private Set<Relationship> relationships;
@@ -116,7 +147,7 @@ public class AttributesToJSON extends AbstractProcessor {
      * Builds the Map of attributes that should be included in the JSON that is emitted from this process.
      *
      * @return
-     *  Map<String, String> of values that are feed to a Jackson ObjectMapper
+     *  Map of values that are feed to a Jackson ObjectMapper
      */
     protected Map<String, String> buildAttributesMapForFlowFile(FlowFile ff, String atrList,
                                                                 boolean includeCoreAttributes,
@@ -198,6 +229,7 @@ public class AttributesToJSON extends AbstractProcessor {
                             }
                         }
                     });
+                    conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), APPLICATION_JSON);
                     session.transfer(conFlowfile, REL_SUCCESS);
                     break;
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/aef0d8fe/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
index 1624c4b..0f9ec26 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToJSON.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.nifi.processors.standard;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -63,8 +80,8 @@ public class TestAttributesToJSON {
         testRunner.run();
 
         //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 
@@ -93,8 +110,8 @@ public class TestAttributesToJSON {
         testRunner.run();
 
         //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 
@@ -123,8 +140,8 @@ public class TestAttributesToJSON {
         testRunner.run();
 
         //Expecting success transition because Jackson is taking care of escaping the bad JSON characters
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
     }
@@ -143,8 +160,8 @@ public class TestAttributesToJSON {
         testRunner.enqueue(ff);
         testRunner.run();
 
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 
@@ -169,8 +186,8 @@ public class TestAttributesToJSON {
         testRunner.enqueue(ff);
         testRunner.run();
 
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeNotExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeNotExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
         testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).assertContentEquals("{}");
@@ -190,8 +207,8 @@ public class TestAttributesToJSON {
         testRunner.enqueue(ff);
         testRunner.run();
 
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 
@@ -218,8 +235,8 @@ public class TestAttributesToJSON {
         testRunner.enqueue(ff);
         testRunner.run();
 
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 
@@ -246,8 +263,8 @@ public class TestAttributesToJSON {
         testRunner.enqueue(ff);
         testRunner.run();
 
-        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0).
-                assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
+        testRunner.getFlowFilesForRelationship(AttributesToJSON.REL_SUCCESS).get(0)
+                .assertAttributeExists(AttributesToJSON.JSON_ATTRIBUTE_NAME);
         testRunner.assertTransferCount(AttributesToJSON.REL_SUCCESS, 1);
         testRunner.assertTransferCount(AttributesToJSON.REL_FAILURE, 0);
 


[34/50] [abbrv] nifi git commit: NIFI-1079 Pull Request functions as described and passes contrib-check. This closes #109.

Posted by ma...@apache.org.
NIFI-1079 Pull Request functions as described and passes contrib-check. This closes #109.


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

Branch: refs/heads/NIFI-730
Commit: 1d97876f82cdd37b2bab5e99ffc170f483c32037
Parents: eb389cf
Author: Bryan Bende <bb...@apache.org>
Authored: Fri Oct 30 14:35:35 2015 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Fri Oct 30 14:36:46 2015 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[21/50] [abbrv] nifi git commit: NIFI-973: Created a Getting Started Guide

Posted by ma...@apache.org.
NIFI-973: Created a Getting Started Guide

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: 4c0cf7d72bdfb0e92159e50d010c3cfa1f264369
Parents: af19053
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Sep 17 13:08:39 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Oct 27 12:01:17 2015 -0400

----------------------------------------------------------------------
 .../src/main/asciidoc/getting-started.adoc      | 754 +++++++++++++++++++
 .../src/main/asciidoc/images/add-processor.png  | Bin 31524 -> 92164 bytes
 nifi-docs/src/main/asciidoc/user-guide.adoc     |   2 +-
 3 files changed, 755 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/4c0cf7d7/nifi-docs/src/main/asciidoc/getting-started.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/getting-started.adoc b/nifi-docs/src/main/asciidoc/getting-started.adoc
new file mode 100644
index 0000000..d6eeaba
--- /dev/null
+++ b/nifi-docs/src/main/asciidoc/getting-started.adoc
@@ -0,0 +1,754 @@
+//
+// 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.
+//
+Getting Started with Apache NiFi
+================================
+Apache NiFi Team <de...@nifi.apache.org>
+:homepage: http://nifi.apache.org
+
+
+Who is This Guide For?
+----------------------
+
+This guide is written for user who have never used, have had limited exposure to, or only accomplished specific tasks within NiFi.
+This guide is not intended to be an exhaustive instruction manual or a reference guide. The
+link:user-guide.html[User Guide] provides a great deal of information
+and is intended to be a much more exhaustive resource and is very useful as a reference guide, as well.
+This guide, in comparison, is intended to provide users with just the information needed in order
+to understand how to work with NiFi in order to quickly and easily build powerful and agile dataflows.
+
+Because some of the information in this guide is applicable only for first-time users while other
+information may be applicable for those who have used NiFi a bit, this guide is broken up into
+several different sections, some of which may not be useful for some readers. Feel free to jump to
+the sections that are most appropriate for you.
+
+This guide does expect that the user has a basic understanding of what NiFi is and does not
+delve into this level of detail. This level of information can be found in the
+link:overview.html[Overview] documentation.
+
+
+
+
+Terminology Used in This Guide
+------------------------------
+
+In order to talk about NiFi, there are a few key terms that readers should be familiar with.
+We will explain those NiFi-specific terms here, at a high level.
+
+
+*FlowFile*: Each piece of "User Data" (i.e., data that the user brings into NiFi for processing and distribution) is
+referred to as a FlowFile. A FlowFile is made up of two parts: Attributes and Content. The Content is the User Data
+itself. Attributes are key-value pairs that are associated with the User Data.
+
+*Processor*: The Processor is the NiFi component that is responsible for creating, sending, receiving, transforming, routing,
+splitting, merging, and processing FlowFiles. It is the most important building block available to NiFi users to build their
+dataflows.
+
+
+Downloading and Installing NiFi
+-------------------------------
+
+NiFi can be downloaded from the link:http://nifi.apache.org/download.html[NiFi Downloads Page]. There are two packaging options
+available: a "tarball" that is tailored more to Linux and a zip file that is more applicable for Windows users. Mac OSX users
+may also use the tarball or can install via Homebrew.
+
+To install via Homebrew, simply run the command `brew install nifi`.
+
+For users who are not running OSX or do not have Homebrew installed, after downloading the version of NiFi that you
+would like to use simply extract the archive to the location that you wish to run the application from.
+
+For information on how to configure the instance of NiFi (for instance, to configure security, data storage
+configuration, or the port that NiFi is running on), see the link:administration-guide.html[Admin Guide].
+
+
+Starting NiFi
+-------------
+
+Once NiFi has been downloaded and installed as described above, it can be started by using the mechanism
+appropriate for your operating system.
+
+=== For Windows Users
+
+For Windows users, navigate to the folder where NiFi was installed. Within this folder is a subfolder
+named `bin`. Navigate to this subfolder and double-click the `run-nifi.bat` file.
+
+This will launch NiFi and leave it running in the foreground. To shut down NiFi, select the window that
+was launched and hold the Ctrl key while pressing C.
+
+
+=== For Linux/Mac OSX users
+
+For Linux and OSX users, use a Terminal window to navigate to the directory where NiFi was installed.
+To run NiFi in the foreground, run `bin/nifi.sh run`. This will leave the application running until
+the user presses Ctrl-C. At that time, it will initiate shutdown of the application.
+
+To run NiFi in the background, instead run `bin/nifi.sh start`. This will initiate the application to
+begin running. To check the status and see if NiFi is currently running, execute the command `bin/nifi.sh status`.
+NiFi can be shutdown by executing the command `bin/nifi.sh stop`.
+
+
+=== Installing as a Service
+
+Currently, installing NiFi as a service is supported only for Linux and Mac OSX users. To install the application
+as a service, navigate to the installation directory in a Terminal window and execute the command `bin/nifi.sh install`
+to install the service with the default name `nifi`. To specify a custom name for the service, execute the command
+with an optional second argument that is the name of the service. For example, to install NiFi as a service with the
+name `dataflow`, use the command `bin/nifi.sh install dataflow`.
+
+Once installed, the service can be started and stopped using the appropriate commands, such as `sudo service nifi start`
+and `sudo service nifi stop`. Additionally, the running status can be checked via `sudo service nifi status`.
+
+
+
+I Started NiFi. Now What?
+-------------------------
+
+Now that NiFi has been started, we can bring up the User Interface (UI) in order to create and monitor our dataflow.
+To get started, open a web browser and navigate to `http://localhost:8080/nifi`. The port can be changed by
+editing the `nifi.properties` file in the NiFi `conf` directory, but the default port is 8080.
+
+This will bring up the User Interface, which at this point is a blank canvas for orchestrating a dataflow:
+
+image:new-flow.png["New Flow"]
+
+Near the top of the UI are a few toolbars that will be very important to create your first dataflow:
+
+image:nifi-toolbar-components.png["Toolbar Components"]
+
+
+=== Adding a Processor
+
+We can now begin creating our dataflow by adding a Processor to our canvas. To do this, drag the Processor icon
+image:iconProcessor.png["Processor"] from the top-left of the screen into the middle of the canvas (the graph paper-like
+background) and drop it there. This will give us a dialog that allows us to choose which Processor we want to add:
+
+image:add-processor.png["Add Processor"]
+
+We have quite a few options to choose from. For the sake of becoming oriented with the system, let's say that we
+just want to bring in files from our local disk into NiFi. When a developer creates a Processor, the developer can 
+assign "tags" to that Processor. These can be thought of as keywords. We can filter by these tags or by Processor
+name by typing into the Filter box in the top-right corner of the dialog. Type in the keywords that you would think
+of when wanting to ingest files from a local disk. Typing in keyword "file", for instance, will provide us a few
+different Processors that deal with files. Filtering by the term "local" will narrow down the list pretty quickly,
+as well. If we select a Processor from the list,
+we will see a brief description of the Processor near the bottom of the dialog. This should tell us exactly what
+the Processor does. The description of the *GetFile* Processor tells us that it pulls data from our local disk
+into NiFi and then removes the local file. We can then double-click the Processor type or select it and choose the
+`Add` button. The Processor will be added to the canvas in the location that it was dropped.
+
+=== Configuring a Processor
+
+Now that we have added the GetFile Processor, we can configure it by right-clicking on the Processor and choosing
+the `Configure` menu item. The provided dialog allows us to configure many different options that can be read about
+in the link:user-guide.html[User Guide], but for the sake of this guide, we will focus on the Properties tab. Once
+the Properties tab has been selected, we are given a list of several different properties that we can configure
+for the Processor. The properties that are available depend on the type of Processor and are generally different
+for each type. Properties that are in bold are required properties. The Processor cannot be started until all required
+properties have been configured. The most important property to configure for GetFile is the directory from which
+to pick up files. If we set the directory name to `./data-in`, this will cause the Processor to start picking up
+any data in the `data-in` subdirectory of the NiFi Home directory. We can choose to configure several different
+Properties for this Processor. If unsure what a particular Property does, we can hover over the help icon (
+image:iconInfo.png["Help"]
+)
+next to the Property Name with the mouse in order to read a description of the property. Additionally, the
+tooltip that is displayed when hovering over the help icon will provide the default value for that property,
+if one exists, information about whether or not the property supports the Expression Language (see the
+<<ExpressionLanguage>> section below), and previously configured values for that property.
+
+In order for this property to be valid, create a directory named `data-in` in the NiFi home directory and then
+click the `OK` button to close the dialog.
+
+
+=== Connecting Processors
+
+Each Processor has a set of defined "Relationships" that it is able to send data to. When a Processor finishes handling
+a FlowFile, it transfers it to one of these Relationships. This allows a user to configure how to handle FlowFiles based
+on the result of Processing. For example, many Processor define two Relationships: `success` and `failure`. Users are
+then able to configure data to be routed through the flow one way if the Processor is able to successfully process
+the data and route the data through the flow in a completely different many if the Processor cannot process the
+data for some reason. Or, depending on the use case, the may simply route both relationships to the same route through
+the flow.
+
+Now that we have added and configured our GetFile processor and applied the configuration, we can see in the
+top-left corner of the Processor an Alert icon (
+image:iconAlert.png[Alert]
+) signaling that the Processor is not in a valid state. Hovering over this icon, we can see that the `success`
+relationship has not been defined. This simply means that we have not told NiFi what to do with the data that the Processor
+transfers to the `success` Relationship.
+
+In order to address this, let's add another Processor that we can connect the GetFile Processor to, by following
+the same steps above. This time, however, we will simply log the attributes that exist for the FlowFile. To do this,
+we will add a LogAttributes Processor.
+
+We can now send the output of the GetFile Processor to the LogAttribute Processor. Hover over the GetFile Processor
+with the mouse and a Connection Icon (
+image:iconConnection.png[Connection]
+) will appear over the middle of the Processor. We can drag this icon from the GetFile Processor to the LogAttribute
+Processor. This gives us a dialog to choose which Relationships we want to include for this connection. Because GetFile
+has only a single Relationship, `success`, it is automatically selected for us.
+
+Clicking on the Settings tab provides a handful of options for configuring how this Connection should behave:
+
+image:connection-settings.png[Connection Settings]
+
+We can give the Connection a name, if we like. Otherwise, the Connection name will be based on the selected Relationships.
+We can also set an expiration for the data. By default, it is set to "0 sec" which indicates that the data should not
+expire. However, we can change the value so that when data in this Connection reaches a certain age, it will automatically
+be deleted (and a corresponding EXPIRE Provenance event will be created).
+
+The backpressure thresholds allow us to specify how full the queue is allowed to become before the source Processor is
+no longer scheduled to run. This allows us to handle cases where one Processor is capable of producing data faster than
+the next Processor is capable of consuming that data. If the backpressure is configured for each Connection along the way,
+the Processor that is bringing data into the system will eventually experience the backpressure and stop bringing in new
+data so that our system has the ability to recover.
+
+Finally, we have the Prioritizers on the right-hand side. This allows us to control how the data in this queue is ordered.
+We can drag Prioritizers from the "Available prioritizers" list to the "Selected prioritizers" list in order to active
+the prioritizer. If multiple prioritizers are activated, they will be evaluated such that the Prioritizer listed first
+will be evaluated first and if two FlowFiles are determined to be equal according to that Prioritizers, the second Prioritizer
+will be used.
+
+For the sake of this discuss, we can simply click `Add`. to add the Connection to our graph. We should now see that the Alert 
+icon has changed to a Stopped icon (
+image:iconStop.png[Stopped]
+). The LogAttribute Processor, however, is now invalid because its `success` Relationship has not been connected to
+anything. Let's address this by signaling that data that is routed to `success` by LogAttribute should be "Auto Terminated,"
+meaning that NiFi should consider the FlowFile's processing complete and "drop" the data. To do this, we configure the
+LogAttribute Processor. On the Settings tab, in the right-hand side we can check the box next to the `success` Relationship
+to Auto Terminate the data. Clicking `OK` will close the dialog and show that both Processors are now stopped.
+
+
+=== Starting and Stopping Processors
+
+At this point, we have two Processors on our graph, but nothing is happening. In order to start the Processors, we can
+click on each one individually and then right-click and choose the `Start` menu item. Alternatively, we can select the first
+Processor, and then hold the Shift key while selecting the other Processor in order to select both. Then, we can
+right-click and choose the `Start` menu item. As an alternative to using the context menu, we can select the Processors and
+then click the Start icon in the toolbar at the top of the screen.
+
+Once started, the icon in the top-left corner of the Processors will change from a stopped icon to a Running icon. We can then
+stop the Processors in the same manner that we started them but using the Stop icon in the toolbar or the Stop menu item
+as opposed to the Start button.
+
+Once a Processor has started, we are not able to configure it anymore. Instead, when we right-click on the Processor, we are
+given the option to view its current configuration. In order to configure a Processor, we must first stop the Processor and
+wait for any tasks that may be executing to finish. The number of tasks currently executing is shown in the top-right
+corner of the Processor, but nothing is shown there if there are currently no tasks.
+
+
+=== Getting More Info for a Processor
+
+With each Processor having the ability to expose multiple different Properties and Relationships, it can become quite
+difficult to remember how all of the different pieces work for each Processor. To address this, you are able to right-click
+on a Processor and choose the `Usage` menu item. This will provide you with the Processor's usage information, such as a
+description of the Processor, the different Relationships that are available, when the different Relationships are used,
+Properties that are exposed by the Processor and their documentation, as well as which FlowFile Attributes (if any) are
+expected on incoming FlowFiles and which Attributes (if any) are added to outgoing FlowFiles.
+
+
+=== Other Components
+
+The toolbar that provides users the ability to drag and drop Processors onto the graph includes several other components
+that can be used to build a dataflow. These components include Input and Output Ports, Funnels, Process Groups, and Remote
+Process Groups. Due to the intended scope of this document, we will not discuss these elements here, but information is
+readily available in the link:user-guide.html#building-dataflow[Building a Dataflow section] of the
+link:user-guide.html[User Guide].
+
+
+
+What Processors are Available
+-----------------------------
+
+In order to create an effective dataflow, the users must understand what types of Processors are available to them.
+NiFi contains many different Processors out of the box. These Processors provide capabilities to ingest data from
+numerous different systems, route, transform, process, split, and aggregate data, and distribute data to many systems.
+
+The number of Processors that are available increases in nearly each release of NiFi. As a result, we will not attempt
+to name each of the Processors that are available, but we will highlight some of the most frequently used Processors,
+categorizing them by their functions.
+
+=== Data Transformation
+- *CompressContent*: Compress or Decompress Content
+- *ConvertCharacterSet*: Convert the character set used to encode the content from one character set to another
+- *EncryptContent*: Encrypt or Decrypt Content
+- *ReplaceText*: Use Regular Expressions to modify textual Content
+- *TransformXml*: Apply an XSLT transform to XML Content
+
+=== Routing and Mediation
+- *ControlRate*: Throttle the rate at which data can flow through one part of the flow
+- *DetectDuplicate*: Monitor for duplicate FlowFiles, based on some user-defined criteria. Often used in conjunction
+	with HashContent
+- *DistributeLoad*: Load balance or sample data by distributing only a portion of data to each user-defined Relationship
+- *MonitorActivity*: Sends a notification when a user-defined period of time elapses without any data coming through a particular
+	point in the flow. Optionally send a notification when dataflow resumes.
+- *RouteOnAttribute*: Route FlowFile based on the attributes that it contains.
+- *ScanAttribute*: Scans the user-defined set of Attributes on a FlowFile, checking to see if any of the Attributes match the terms
+  found in a user-defined dictionary.
+- *RouteOnContent*: Search Content of a FlowFile to see if it matches any user-defined Regular Expression. If so, the FlowFile is
+  routed to the configured Relationship.
+- *ScanContent*: Search Content of a FlowFile for terms that are present in a user-defined dictionary and route based on the
+	presence or absence of those terms. The dictionary can consist of either textual entries or binary entries.
+- *ValidateXml*: Validation XML Content against an XML Schema; routes FlowFile based on whether or not the Content of the FlowFile
+	is valid according to the user-defined XML Schema.
+
+=== Database Access
+- *ConvertJSONToSQL*: Convert a JSON document into a SQL INSERT or UPDATE command that can then be passed to the PutSQL Processor
+- *ExecuteSQL*: Executes a user-defined SQL SELECT command, writing the results to a FlowFile in Avro format
+- *PutSQL*: Updates a database by executing the SQL DDM statement defined by the FlowFile's content
+
+[[AttributeExtraction]]
+=== Attribute Extraction
+- *EvaluateJsonPath*: User supplies JSONPath Expressions (Similar to XPath, which is used for XML parsing/extraction), and these Expressions
+  are then evaluated against the JSON Content to either replace the FlowFile Content or extract the value into the user-named Attribute.
+- *EvaluateXPath*: User supplies XPath Expressions, and these Expressions are then evaluated against the XML Content to either
+  replace the FlowFile Content or extract the value into the user-named Attribute.
+- *EvaluateXQuery*: User supplies an XQuery query, and this query is then evaluated against the XML Content to either replace the FlowFile
+  Content or extract the value into the user-named Attribute.
+- *ExtractText*: User supplies one or more Regular Expressions that are then evaluated against the textual content of the FlowFile, and the
+  values that are extracted are then added as user-named Attributes.
+- *HashAttribute*: Performs a hashing function against the concatenation of a user-defined list of existing Attributes.
+- *HashContent*: Performs a hashing function against the content of a FlowFile and adds the hash value as an Attribute.
+- *IdentifyMimeType*: Evaluates the content of a FlowFile in order to determine what type of file the FlowFile encapsulates. This Processor is
+  capable of detecting many different MIME Types, such as images, word processor documents, text, and compression formats just to name
+	a few.
+- *UpdateAttribute*: Adds or updates any number of user-defined Attributes to a FlowFile. This is useful for adding statically configured values,
+  as well as deriving Attribute values dynamically by using the Expression Language. This processor also provides an "Advanced User Interface,"
+	allowing users to update Attributes conditionally, based on user-supplied rules.
+
+=== System Interaction
+- *ExecuteProcess*: Runs the user-defined Operating System command. The Process's StdOut is redirected such that the content that is written
+  to StdOut becomes the content of the outbound FlowFile. This Processor is a Source Processor - its output is expected to generate a new FlowFile,
+	and the system call is expected to receive no input. In order to provide input to the process, use the ExecuteStreamCommand Processor.
+
+- *ExecuteStreamCommand*: Runs the user-defined Operating System command. The contents of the FlowFile are optionally streamed to the StdIn
+  of the process. The content that is written to StdOut becomes the content of hte outbound FlowFile. This Processor cannot be used a Source Processor -
+	it must be fed incoming FlowFiles in order to perform its work. To perform the same type of functionality with a Source Processor, see the
+	ExecuteProcess Processor.
+
+=== Data Ingestion
+- *GetFile*: Streams the contents of a file from a local disk (or network-attached disk) into NiFi and then deletes the original file. This
+  Processor is expected to move the file from one location to another location and is not to be used for copying the data.
+- *GetFTP*: Downloads the contents of a remote file via FTP into NiFi and then deletes the original file. This Processor is expected to move
+  the data from one location to another location and is not to be used for copying the data.
+- *GetSFTP*: Downloads the contents of a remote file via SFTP into NiFi and then deletes the original file. This Processor is expected to move
+  the data from one location to another location and is not to be used for copying the data.
+- *GetJMSQueue*: Downloads a message from a JMS Queue and creates a FlowFile based on the contents of the JMS message. The JMS Properties are
+  optionally copied over as Attributes, as well.
+- *GetJMSTopic*: Downloads a message from a JMS Topic and creates a FlowFile based on the contents of the JMS message. The JMS Properties are
+  optionally copied over as Attributes, as well. This Processor supports both durable and non-durable subscriptions.
+- *GetHTTP*: Downloads the contents of a remote HTTP- or HTTPS-based URL into NiFi. The Processor will remember the ETag and Last-Modified Date
+  in order to ensure that the data is not continually ingested.
+- *ListenHTTP*: Starts an HTTP (or HTTPS) Server and listens for incoming connections. For any incoming POST request, the contents of the request
+  are written out as a FlowFile, and a 200 response is returned.
+- *ListenUDP*: Listens for incoming UDP packets and creates a FlowFile per packet or per bundle of packets (depending on configuration) and
+  emits the FlowFile to the 'success' relationship.
+- *GetHDFS*: Monitors a user-specified directory in HDFS. Whenever a new file enters HDFS, it is copied into NiFi and deleted from HDFS. This
+  Processor is expected to move the file from one location to another location and is not to be used for copying the data. This Processor is also
+	expected to be run On Primary Node only, if run within a cluster. In order to copy the data from HDFS and leave it in-tact, or to stream the data
+	from multiple nodes in the cluster, see the ListHDFS Processor.
+- *ListHDFS* / *FetchHDFS*: ListHDFS monitors a user-specified directory in HDFS and emits a FlowFile containing the filename for each file that it
+  encounters. It then persists this state across the entire NiFi cluster by way of a Distributed Cache. These FlowFiles can then be fanned out across
+	the cluster and sent to the FetchHDFS Processor, which is responsible for fetching the actual content of those files and emitting FlowFiles that contain
+	the content fetched from HDFS.
+- *FetchS3Object*: Fetches the contents of an object from the Amazon Web Services (AWS) Simple Storage Service (S3). The outbound FlowFile contains the contents
+  received from S3.
+- *GetKafka*: Consumes messages from Apache Kafka. The messages can be emitted as a FlowFile per message or can be batched together using a user-specified
+  delimiter.
+- *GetMongo*: Executes a user-specified query against MongoDB and writes the contents to a new FlowFile.
+- *GetTwitter*: Allows Users to register a filter to listen to the Twitter "garden hose" or Enterprise endpoint, creating a FlowFile for each tweet
+  that is received.
+
+=== Data Egress / Sending Data
+- *PutEmail*: Sends an E-mail to the configured recipients. The content of the FlowFile is optionally sent as an attachment.
+- *PutFile*: Writes the contents of a FlowFile to a directory on the local (or network attached) file system.
+- *PutFTP*: Copies the contents of a FlowFile to a remote FTP Server.
+- *PutSFTP*: Copies the contents of a FlowFile to a remote SFTP Server.
+- *PutJMS*: Sends the contents of a FlowFile as a JMS message to a JMS broker, optionally adding JMS Properties based on Attributes.
+- *PutSQL*: Executes the contents of a FlowFile as a SQL DDL Statement (INSERT, UPDATE, or DELETE). The contents of the FlowFile must be a valid
+  SQL statement. Attributes can be used as parameters so that the contents of the FlowFile can be parameterized SQL statements in order to avoid
+	SQL injection attacks.
+- *PutKafka*: Sends the contents of a FlowFile to Kafka as a message. The FlowFile can be sent as a single message or a delimiter, such as a
+  new-line can be specified, in order to send many messages for a single FlowFile.
+- *PutMongo*: Sends the contents of a FlowFile to Mongo as an INSERT or an UPDATE.
+
+=== Splitting and Aggregation
+- *SplitText*: SplitText takes in a single FlowFile whose contents are textual and splits it into 1 or more FlowFiles based on the configured
+  number of lines. For example, the Processor can be configured to split a FlowFile into many FlowFiles, each of which is only 1 line.
+- *SplitJson*: Allows the user to split a JSON object that is comprised of an array or many child objects into a FlowFile per JSON element.
+- *SplitXml*: Allows the user to split an XML message into many FlowFiles, each containing a segment of the original. This is generally used when
+  several XML elements have been joined together with a "wrapper" element. This Processor then allows those elements to be split out into individual
+	XML elements.
+- *UnpackContent*: Unpacks different types of archive formats, such as ZIP and TAR. Each file within the archive is then transferred as a single
+  FlowFile.
+- *MergeContent*: This Processor is responsible for merging many FlowFiles into a single FlowFile. The FlowFiles can be merged by concatenating their
+  content together along with optional header, footer, and demarcator, or by specifying an archive format, such as ZIP or TAR. FlowFiles can be binned
+	together based on a common attribute, or can be "defragmented" if they were split apart by some other Splitting process. The minimum and maximum
+	size of each bin is user-specified, based on number of elements or total size of FlowFiles' contents, and an optional Timeout can be assigned as well
+	so that FlowFiles will only wait for their bin to become full for a certain amount of time.
+- *SegmentContent*: Segments a FlowFile into potentially many smaller FlowFiles based on some configured data size. The splitting is not performed
+  against any sort of demarcator but rather just based on byte offsets. This is used before transmitting FlowFiles in order to provide lower latency
+	by sending many different pieces in parallel. On the other side, these FlowFiles can then be reassembled by the MergeContent processor using the
+	Defragment mode.
+- *SplitContent*: Splits a single FlowFile into potentially many FlowFiles, similarly to SegmentContent. However, with SplitContent, the splitting
+  is not performed on arbitrary byte boundaries but rather a byte sequence is specified on which to split the content.
+
+=== HTTP
+- *GetHTTP*: Downloads the contents of a remote HTTP- or HTTPS-based URL into NiFi. The Processor will remember the ETag and Last-Modified Date
+  in order to ensure that the data is not continually ingested.
+- *ListenHTTP*: Starts an HTTP (or HTTPS) Server and listens for incoming connections. For any incoming POST request, the contents of the request
+  are written out as a FlowFile, and a 200 response is returned.
+- *InvokeHTTP*: Performs an HTTP Request that is configured by the user. This Processor is much more versatile than the GetHTTP and PostHTTP
+  but requires a bit more configuration. This Processor cannot be used as a Source Processor and is required to have incoming FlowFiles in order
+	to be triggered to perform its task.
+- *PostHTTP*: Performs an HTTP POST request, sending the contents of the FlowFile as the body of the message. This is often used in conjunction
+  with ListenHTTP in order to transfer data between two different instances of NiFi in cases where Site-to-Site cannot be used (for instance,
+	when the nodes cannot access each other directly and are able to communicate through an HTTP proxy).
+- *HandleHttpRequest* / *HandleHttpResponse*: The HandleHttpRequest Processor is a Source Processor that starts an embedded HTTP(S) server
+  similarly to ListenHTTP. However, it does not send a response to the client. Instead, the FlowFile is sent out with the body of the HTTP request
+	as its contents and attributes for all of the typical Servlet parameters, headers, etc. as Attributes. The HandleHttpResponse then is able to
+	send a response back to the client after the FlowFile has finished being processed. These Processors are always expected to be used in conjunction
+	with one another and allow the user to visually create a Web Service within NiFi. This is particularly useful for adding a front-end to a non-web-
+  based protocol or to add a simple web service around some functionality that is already performed by NiFi, such as data format conversion.
+
+=== Amazon Web Services
+- *FetchS3Object*: Fetches the content of an object stored in Amazon Simple Storage Service (S3). The content that is retrieved from S3
+  is then written to the content of the FlowFile.
+- *PutS3Object*: Writes the contents of a FlowFile to an Amazon S3 object using the configured credentials, key, and bucket name.
+- *PutSNS*: Sends the contents of a FlowFile as a notification to the Amazon Simple Notification Service (SNS).
+- *GetSQS*: Pulls a message from the Amazon Simple Queuing Service (SQS) and writes the contents of the message to the content of the FlowFile.
+- *PutSQS*: Sends the contents of a FlowFile as a message to the Amazon Simple Queuing Service (SQS).
+- *DeleteSQS*: Deletes a message from the Amazon Simple Queuing Service (SQS). This can be used in conjunction with the GetSQS in order to receive
+  a message from SQS, perform some processing on it, and then delete the object from the queue only after it has successfully completed processing.
+
+
+Working With Attributes
+-----------------------
+Each FlowFile is created with several Attributes, and these Attributes will change over the life of
+the FlowFile. The concept of a FlowFile is extremely powerful and provides three primary benefits.
+First, it allows the user to make routing decisions in the flow so that FlowFiles that meeting some criteria
+can be handled differently than other FlowFiles. This is done using the RouteOnAttribute and similar Processors.
+
+Secondly, Attributes are used in order to configure Processors in such a way that the configurationg of the
+Processor is dependent on the data itself. For instance, the PutFile Processor is able to use the Attributes in order
+to know where to store each FlowFile, while the directory and filename Attributes may be different for each FlowFile.
+
+Finally, the Attributes provide extremely valuable context about the data. This is useful when reviewing the Provenance
+data for a FlowFile. This allows the user to search for Provenance data that match specific criteria, and it also allows
+the user to view this context when inspecting the details of a Provenance Event. By doing this, the user is then able
+to gain valuable insight as to why the data was processed one way or another, simply by glancing at this context that is
+carried along with the content.
+
+=== Common Attributes
+
+Each FlowFile has a minimum set of Attributes:
+
+	- *filename*: A filename that can be used to store the data to a local or remote file system
+	- *path*: The name of a directory that can be used to store the data to a local or remote file system
+	- *uuid*: A Universally Unique Identifier that distinguishes the FlowFile from other FlowFiles in the system.
+	- *entryDate*: The date and time at which the FlowFile entered the system (i.e., was created). The value of this
+		attribute is a number that represents the number of milliseconds since midnight, Jan. 1, 1970 (UTC).
+	- *lineageStartDate*: Any time that a FlowFile is cloned, merged, or split, this results in a "child" FlowFile being
+		created. As those children are then cloned, merged, or split, a chain of ancestors is built. This value represents
+		the date and time at which the oldest ancestor entered the system. Another way to think about this is that this
+		attribute represents the latency of the FlowFile through the system. The value is a number that represents the number
+		of milliseconds since midnight, Jan. 1, 1970 (UTC).
+	- *fileSize*: This attribute represents the number of bytes taken up by the FlowFile's Content.
+
+Note that the `uuid`, `entryDate`, `lineageStartDate`, and `fileSize` attributes are system-generated and cannot be changed.
+
+=== Extracting Attributes
+
+NiFi provides several different Processors out of the box for extracting Attributes from FlowFiles. A list of commonly used
+Processors for this purpose can be found above in the <<AttributeExtraction>> section. This is a very common use case for building
+custom Processors, as well. Many Processors are written to understand a specific data format and extract pertinent information from
+a FlowFile's content, creating Attributes to hold that information, so that decisions can then be made about how to route or
+process the data.
+
+=== Adding User-Defined Attributes
+
+In addition to having Processors that are able to extract particular pieces of information from FlowFile content into Attributes,
+it is also common for users to want to add their own user-defined Attributes to each FlowFile at a particular place in the flow.
+The UpdateAttribute Processor is designed specifically for this purpose. Users are able to add a new property to the Processor
+in the Configure dialog by clicking the "New Property" button in the top-right corner of the Properties tab. The user is then
+prompted to enter the name of the property and then a value. For each FlowFile that is processed by this UpdateAttribute
+Processor, an Attribute will be added for each user-defined property. The name of the Attribute will be the same as the name of
+the property that was added. The value of the Attribute will be the same as the value of the property.
+
+The value of the property may contain the Expression Language, as well. This allows Attributes to be modified or added
+based on other Attributes. For example, if we want to prepend the hostname that is processing a file as well as the date to
+a filename, we could do this by adding a property with the name `filename` and the value `${hostname()}-${now():format('yyyy-dd-MM')}-${filename}`.
+While this may seem confusing at first, the section below on <<ExpressionLanguage>> will help to clear up what is going on
+here.
+
+In addition to always adding a defined set of Attributes, the UpdateAttribute Processor has an Advanced UI that allows the user
+to configure a set of rules for which Attributes should be added when. To access this capability, in the Configure dialog's
+Properties tab, click the `Advanced...` button at the bottom of the dialog. This will provide a UI that is tailored specifically
+to this Processor, rather than the simple Properties table that is provided for all Processors. Within this UI, the user is able
+to configure a rules engine, essentially, specifying rules that must match in order to have the configured Attributes added
+to the FlowFile.
+
+=== Routing on Attributes
+
+One of the most powerful features of NiFi is the ability to route FlowFiles based on their Attributes. The primary mechanism
+for doing this is the RouteOnAttribute Processor. This Processor, like UpdateAttribute, is configured by adding user-defined properties.
+Any number of properties can be added by clicking the "New Property" icon in the top-right corner of the Properties tab in the
+Processor's Configure dialog.
+
+Each FlowFile's Attributes will be compared against the configured properties to determine whether or not the FlowFile meets the
+specified criteria. The value of each property is expected to be an Expression Language expression and return a boolean value.
+For more on the Expression Language, see the <<ExpressionLanguage>> section below.
+
+After evaluating the Expression Language expressions provided against the FlowFile's Attributes, the Processor determines how to
+route the FlowFile based on the Routing Strategy selected. The most common strategy is the "Route to Property name" strategy. With this
+strategy selected, the Processor will expose a Relationship for each property configured. If the FlowFile's Attributes satisfy the given
+expression, a copy of the FlowFile will be routed to the corresponding Relationship. For example, if we had a new property with the name
+"begins-with-r" and the value "${filename:startsWith(\'r')}" then any FlowFile whose filename starts with the letter 'r' will be routed
+to that Relationship. All other FlowFiles will be routed to 'unmatched'.
+
+
+[[ExpressionLanguage]]
+=== Expression Language / Using Attributes in Property Values
+
+As we extract Attributes from FlowFiles' contents and add user-defined Attributes, they don't do us much good as an operator unless
+we have some mechanism by which we can use them. The NiFi Expression Language allows us to access and manipulate FlowFile Attribute
+values as we configure our flows. Not all Processor properties allow the Expression Language to be used, but many do. In order to
+determine whether or not a property supports the Expression Language, a user can hover over the Help icon (
+icon:iconInfo.png["Help Icon"]
+) in the Properties tab of the Processor Configure dialog. This will provide a tooltip that shows a description of the property, the
+default value, if any, and whether or not the property supports the Expression Language.
+
+For properties that do support the Expression Language, it is used by adding an expression within the opening `${` tag and the closing
+`}` tag. An expression can be as simple as an attribute name. For example, to reference the `uuid` Attribute, we can simply use the
+value `${uuid}`. If the Attribute name begins with any character other than a letter, or if it contians a character other than
+a number, a letter, a period (.), or an underscore (_), the Attribute name will need to be quoted. For example, `${My Attribute Name}`
+will be invalid, but `${'My Attribute Name'}` will refer to the Attribute `My Attribute Name`.
+
+In addition to referencing Attribute values, we can perform a number of functions and comparisons on those Attributes. For example,
+if we want to check if the `filename` attribute contains the letter 'r' without paying attention to case (upper case or lower case),
+we can do this by using the expression `${filename:toLower():contains('r')}`. Note here that the functions are separated by colons.
+We can chain together any number of functions to build up more complex expressions. It is also important to understand here that even
+though we are calling `filename:toLower()`, this does not alter the value of the `filename` Attribute in anyway but rather just gives
+us a new value to work with.
+
+We can also embed one expression within another. For example, if we wanted to compare the value of the `attr1` Attribute to
+the value of the `attr2` Attribute, we can do this with the following expression: `${attr1:equals( ${attr2} )}`.
+
+The Expression Language contains many different functions that can be used in order to perform the tasks needed for routing and manipulating
+Attributes. Functions exist for parsing and manipulating strings, comparing string and numeric values, manipulating and replacing values,
+and comparing values. A full explanation of the different functions available is out of scope of this document, but the
+link:expression-language-guide.html[Expression Language Guide] provides far greater detail for each of the functions.
+
+In addition, this Expression Language guide is built in to the application so that users are able to easily see which functions are available
+and see their documentation while typing. When setting the value of a property that supports the Expression Language, if the cursor is within
+the Expression Language start and end tags, pressing Ctrl + Space on the keyword will provide a popup of all of the available functions and
+will provide auto-complete functionality. Clicking on or using the keyboard to navigate to one of the functions listed in the popup will
+cause a tooltip to show, which explains what the function does, the arguments that it expects, and the return type of the function.
+
+
+Working With Templates
+----------------------
+
+As we use Processors to build more and more complex dataflows in NiFi, we often will find that we string together the same sequence
+of Processors to perform some task. This can become tedious and inefficient. To address this, NiFi provides a concept of Templates.
+A template can be thought of as a reusable sub-flow. To create a template, follow these steps:
+
+- Select the components to include in the template. We can select multiple components by clicking on the first component and then holding
+  the Shift key while selecting additional components (to include the Connections between those components), or by holding the Shift key
+	while dragging a box around the desired components on the canvas.
+- Select the Create Template Icon (
+	image:iconTemplate.png[Template Icon]
+	) from the middle toolbar at the top of the screen.
+- Provide a name and optionally comments about the template.
+- Click the Create button.
+
+Once we have created a template, we can now use it as a building block in our flow, just as we would a Processor. To do this, we will
+click and drag the Template icon from the left-most toolbar onto our canvas. We can then choose the template that we would like to add
+or our canvas and click the Add button.
+
+Finally, we have the ability to manage our templates by using the Template Management dialog. To access this dialog, click the Template
+icon in the top-right toolbar. From here, we can see which templates exist and filter the templates to find the templates of interest.
+On the right-hand side of the table is icon to Export, or Download, the template as an XML file. This can then be provided to others so
+that they can use your template.
+
+To import a template into your NiFi instance, click the Browse button in the top-right corner of the dialog and navigate to the file on
+your computer. Then click the Import button. The template will now show up in your table, and you can drag it onto your canvas as you would
+any other template that you have created.
+
+There are a few important notes to remember when working with templates:
+- Any properties that are identified as being Sensitive Properties (such as a password that is configured in a Processor) will not be added
+  to the template. These sensitive properties will have to be populated each time that the template is added to the canvas.
+- If a component that is included in the template references a Controller Service, the Controller Service will also be added to the template.
+  This means that each time that the template is added to the graph, it will create a copy of the Controller Service.
+
+
+Monitoring NiFi
+---------------
+
+As data flows through your dataflow in NiFi, it is important to understand how well your system is performing in order to assess if you
+will require more resources and in order to assess the health of your current resources. NiFi provides a few mechanisms for monitoring
+your system.
+
+=== Status Bar
+
+Near the top of the NiFi screen is a blue bar that is referred to as the Status Bar. It contains a few important statistics about the current
+health of NiFi. The number of Active Threads can indicate how hard NiFi is currently working, and the Queued stat indicates how many FlowFiles
+are currently queued across the entire flow, as well as the total size of those FlowFiles.
+
+If the NiFi instance is in a cluster, we will also see an indicator here telling us how many nodes are in the cluster and how many are currently
+connected. In this case, the number of active threads and the queue size are indicative of all the sum of all nodes that are currently connected.
+
+=== Component Statistics
+
+Each Processor, Process Group, and Remote Process Group on the canvas provides several statistics about how much data has been processed
+by the component. These statistics provide information about how much data has been processed in the past five minutes. This is a rolling
+window and allows us to see things like the number of FlowFiles that have been consumed by a Processor, as well as the number of FlowFiles
+that have been emitted by the Processor.
+
+The connections between Processors also expose the number of items that are currently queued.
+
+It may also be valuable to see historical values for these metrics and, if clustered, how the different nodes compare to one another.
+In order to see this information, we can right-click on a component and choose the Status menu item. This will show us a graph that spans
+the time since NiFi was started, or up to 24 hours, whichever is less. The amount of time that is shown here can be extended or reduced
+by changing the configuration in the properties file.
+
+In the top-right corner is a drop-down that allows the user to select which metric they are viewing. The graph on the bottom allows the
+user to select a smaller portion of the graph to zoom in.
+
+
+=== Bulletins
+
+In addition to the statistics provided by each component, as a user we will want to know if any problems occur. While we could monitor the
+logs for anything interesting, it is much more convenient to have notifications pop up on the screen. If a Processor logs
+anything as a WARNING or ERROR, we will see a "Bulletin Indicator" show up in the top-left-hand corner of the Processor. This indicator
+looks like a sticky note and will be shown for five minutes after the event occurs. Hovering over the bulletin provides information about
+what happened so that the user does not have to sift through log messages to find it. If in a cluster, the bulletin will also indicate which
+node in the cluster emitted the bulletin. We can also change the log level at which bulletins will occur in the Settings tab of the Configure
+dialog for a Processor.
+
+If the framework emits a bulletin, we will also see this bulletin indicator occur in the Status Bar at the top of the screen.
+The right-most icon in the NiFi Toolbar is the Bulletin Board icon. Clicking this icon will take us to the bulletin board where
+we can see all bulletins that occur across the NiFi instance and can filter based on the component, the message, etc.
+
+
+Data Provenance
+---------------
+
+NiFi keeps a very granular level of detail about each piece of data that it ingests. As the data is processed through
+the system and is transformed, routed, split, aggregated, and distributed to other endpoints, this information is
+all stored within NiFi's Provenance Repository. In order to search and view this information, we can click the Data Provenance icon (
+image:iconProvenance.png[Data Provenance, width=28]) in the top-right corner of the canvas. This will provide us a table that lists
+the Provenance events that we have searched for:
+
+image:provenance-table.png[Provenance Table]
+
+Initially, this table is populated with the most recent 1,000 Provenance Events that have occurred (though it may take a few
+seconds for the information to be processed after the events occur). From this dialog, there is a Search button that allows the
+user to search for events that happened by a particular Processor, for a particular FlowFile by filename or UUID, or several other
+fields. The `nifi.properties` file provides the ability to configure which of these properties are indexed, or made searchable.
+Additionally, the properties file also allows you to choose specific FlowFile Attributes that will be indexed. As a result, you can
+choose which Attributes will be important to your specific dataflows and make those Attributes searchable.
+
+[[EventDetails]]
+=== Event Details
+Once we have performed our search, our table will be populated only with the events that match the search criteria. From here, we
+can choose the Info icon (
+image:iconInfo.png[Info Icon]
+) on the left-hand side of the table to view the details of that event:
+
+image:event-details.png[Event Details]
+
+From here, we can see exactly when that event occurred, which FlowFile the event affected, which component (Processor, etc.) performed the event,
+how long the event took, and the overall time that the data had been in NiFi when the event occurred (total latency).
+
+The next tab provides a listing of all Attributes that existed on the FlowFile at the time that the event occurred:
+
+image:event-attributes.png[Event Attributes]
+
+From here, we can see all the Attributes that existed on the FlowFile when the event occurred, as well as the previous values for those
+Attributes. This allows us to know which Attributes changed as a result of this event and how they changed. Additionally, in the right-hand
+corner is a checkbox that allows the user to see only those Attributes that changed. This may not be particularly useful if the FlowFile has
+only a handful of Attributes, but can be very helpful when a FlowFile has hundreds of Attributes.
+
+This is very important, because it allows the user to understand the exactly context in which the FlowFile was processed. This is very helpful
+to understand 'why' the FlowFile was processed the way that it was, especially when the Processor was configured using the Expression Language.
+
+Finally, we have the Content tab:
+
+image:event-content.png[Event Content]
+
+This tab provides us information about where in the Content Repository the FlowFile's content was stored. If the event modified the content
+of the FlowFile, we will see the 'before' and 'after' content claims. We are then given the option to Download the content or to View the
+content within NiFi itself, if the data format is one that NiFi understands how to render.
+
+Additionally, there is 'Replay' button that allows the user to re-insert the FlowFile into the flow and re-process it from exactly the point
+at which the event happened. This provides a very powerful mechanism, as we are able to modify our flow in real time, re-process a FlowFile,
+and then view the results. If they are not as expected, we can modify the flow again, and re-process the FlowFile again. We are able to perform
+this iterative development of the flow until it is processing the data exactly as intended.
+
+=== Lineage Graph
+
+In addition to viewing the details of a Provenance event, we can also view the lineage of the FlowFile involved by clicking on the Lineage Icon (
+image:iconLineage.png[Lineage]
+) from the table view.
+
+This provides us with a graphical representation of exactly what happened to that piece of data as it traversed the system:
+
+image:lineage-graph-annotated.png[Lineage Graph]
+
+From here, we can right-click on any of the events represented and click the "View Details" menu item to see the <<EventDetails>>.
+This graphical representation shows us exactly which events occurred to the data. There are a view "special" event types to be
+aware of. If we see a JOIN, FORK, or CLONE event, we can right-click and choose to Find Parents or Expand. This allows us to
+see the lineage of parent FlowFiles and children FlowFiles that were created as well.
+
+The slider in the bottom-left corner allows us to see the time at which these events occurred. By sliding it left and right, we can
+see which events introduced latency into the system so that we have a very good understanding of where in our system we may need to
+provide more resources, such as the number of Concurrent Tasks for a Processor. Or it may reveal, for example, that most of the latency
+was introduced by a JOIN event, in which we were waiting for more FlowFiles to join together. In either case, the ability to easily
+see where this is occurring is a very powerful feature that will help users to understand how the enterprise is operating.
+
+
+Where To Go For More Information
+--------------------------------
+
+The NiFi community has built up a significant amount of documentation on how to use the software. The following guides are available, in
+addition to this Getting Started Guide:
+
+- link:overview.html[Apache NiFi Overview] - Provides an overview of what Apache NiFi is, what it does, and why it was created.
+- link:user-guide.html[Apache NiFi User Guide] - A fairly extensive guide that is often used more as a Reference Guide, as it has pretty
+  lengthy in discussing all of the different components that comprise the application. This guide is written with the NiFi Operator as its
+	audience. It provides information on each of the different components available in NiFi and explains how to use the different features
+	provided by the application.
+- link:administration-guide.html[Administration Guide] - A guide for setting up and administering Apache NiFi for production environments.
+  This guide provides information about the different system-level settings, such as setting up clusters of NiFi and securing access to the
+	web UI and data.
+- link:expression-language-guide.html[Express Language Guide] - A far more exhaustive guide for understanding the Expression Language than
+  is provided above. This guide is the definitive documentation for the NiFi Expression Language. It provides an introduction to the EL
+	and an explanation of each function, its arguments, and return types as well as providing examples.
+- link:developer-guide.html[Developer's Guide] - While not an exhaustive guide to All Things NiFi Development, this guide does provide a
+  comprehensive overview of the different API's available and how they should be used. In addition, it provides Best Practices for developing
+	NiFi components and common Processor idioms to help aid in understanding the logic behind many of the existing NiFi components.
+- link:https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide[Contributor's Guide] - A guide for explaining how to contribute
+  work back to the Apache NiFi community so that others can make use of it.
+
+Several blog postings have also been added to the Apache NiFi blog site:
+link:https://blogs.apache.org/nifi/[https://blogs.apache.org/nifi/]
+
+In addition to the blog and guides provided here, you can browse the different
+link:https://nifi.apache.org/mailing_lists.html[NiFi Mailing Lists] or send an e-mail to one of the mailing lists at
+link:mailto:users@nifi.apache.org[users@nifi.apache.org] or
+link:mailto:dev@nifi.apache.org[dev@nifi.apache.org].
+
+Many of the members of the NiFi community are also available on Twitter and actively monitor for tweets that mention @apachenifi.

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c0cf7d7/nifi-docs/src/main/asciidoc/images/add-processor.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/add-processor.png b/nifi-docs/src/main/asciidoc/images/add-processor.png
index a043f38..3d65e72 100644
Binary files a/nifi-docs/src/main/asciidoc/images/add-processor.png and b/nifi-docs/src/main/asciidoc/images/add-processor.png differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/4c0cf7d7/nifi-docs/src/main/asciidoc/user-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc
index b793af4..d914650 100644
--- a/nifi-docs/src/main/asciidoc/user-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/user-guide.adoc
@@ -155,7 +155,7 @@ image::status-bar.png["NiFi Status Bar"]
 
 
 
-
+[[building-dataflow]]
 Building a DataFlow
 -------------------
 


[03/50] [abbrv] nifi git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/nifi

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


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

Branch: refs/heads/NIFI-730
Commit: 8a8006085190aae1125c883f2aab16a4ada9beb8
Parents: 0636f0e f2c4f2d
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Oct 25 11:11:49 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Oct 25 11:11:49 2015 -0400

----------------------------------------------------------------------
 .../hadoop/AbstractHadoopProcessor.java         | 92 +++++++++++++++++---
 1 file changed, 79 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[26/50] [abbrv] nifi git commit: NIFI-986 Refactoring of action classes from nifi-user-actions to have interfaces in nifi-api, and adding getFlowChanges to EventAccess - Fixing empty java docs and adding sort by id asc to the history query - Changing use

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeExtensionDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeExtensionDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeExtensionDetails.java
new file mode 100644
index 0000000..068ce20
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeExtensionDetails.java
@@ -0,0 +1,35 @@
+/*
+ * 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.action.component.details;
+
+/**
+ *
+ */
+public class FlowChangeExtensionDetails implements ExtensionDetails {
+
+    private String type;
+
+    @Override
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeRemoteProcessGroupDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeRemoteProcessGroupDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeRemoteProcessGroupDetails.java
new file mode 100644
index 0000000..c38690e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/FlowChangeRemoteProcessGroupDetails.java
@@ -0,0 +1,35 @@
+/*
+ * 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.action.component.details;
+
+/**
+ *
+ */
+public class FlowChangeRemoteProcessGroupDetails implements RemoteProcessGroupDetails {
+
+    private String uri;
+
+    @Override
+    public String getUri() {
+        return uri;
+    }
+
+    public void setUri(String uri) {
+        this.uri = uri;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
deleted file mode 100644
index 551f70a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/RemoteProcessGroupDetails.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.action.component.details;
-
-/**
- *
- */
-public class RemoteProcessGroupDetails extends ComponentDetails {
-
-    private String uri;
-
-    public String getUri() {
-        return uri;
-    }
-
-    public void setUri(String uri) {
-        this.uri = uri;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ActionDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ActionDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ActionDetails.java
deleted file mode 100644
index 7c0903d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ActionDetails.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.action.details;
-
-import java.io.Serializable;
-
-/**
- *
- */
-public abstract class ActionDetails implements Serializable {
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
deleted file mode 100644
index 7e20a03..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConfigureDetails.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.action.details;
-
-/**
- *
- */
-public class ConfigureDetails extends ActionDetails {
-
-    private String name;
-    private String previousValue;
-    private String value;
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    public void setPreviousValue(String previousValue) {
-        this.previousValue = previousValue;
-    }
-
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConnectDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
deleted file mode 100644
index 2b2ebaa..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/ConnectDetails.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.action.details;
-
-import org.apache.nifi.action.Component;
-
-/**
- *
- */
-public class ConnectDetails extends ActionDetails {
-
-    private String sourceId;
-    private String sourceName;
-    private Component sourceType;
-    private String relationship;
-    private String destinationId;
-    private String destinationName;
-    private Component destinationType;
-
-    public String getSourceId() {
-        return sourceId;
-    }
-
-    public void setSourceId(String sourceId) {
-        this.sourceId = sourceId;
-    }
-
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public void setSourceName(String sourceName) {
-        this.sourceName = sourceName;
-    }
-
-    public Component getSourceType() {
-        return sourceType;
-    }
-
-    public void setSourceType(Component sourceType) {
-        this.sourceType = sourceType;
-    }
-
-    public String getDestinationId() {
-        return destinationId;
-    }
-
-    public void setDestinationId(String destinationId) {
-        this.destinationId = destinationId;
-    }
-
-    public String getDestinationName() {
-        return destinationName;
-    }
-
-    public void setDestinationName(String destinationName) {
-        this.destinationName = destinationName;
-    }
-
-    public Component getDestinationType() {
-        return destinationType;
-    }
-
-    public void setDestinationType(Component destinationType) {
-        this.destinationType = destinationType;
-    }
-
-    public String getRelationship() {
-        return relationship;
-    }
-
-    public void setRelationship(String relationship) {
-        this.relationship = relationship;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConfigureDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConfigureDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConfigureDetails.java
new file mode 100644
index 0000000..568fe71
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConfigureDetails.java
@@ -0,0 +1,55 @@
+/*
+ * 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.action.details;
+
+/**
+ *
+ */
+public class FlowChangeConfigureDetails implements ConfigureDetails {
+
+    private String name;
+    private String previousValue;
+    private String value;
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    @Override
+    public String getPreviousValue() {
+        return previousValue;
+    }
+
+    public void setPreviousValue(String previousValue) {
+        this.previousValue = previousValue;
+    }
+
+    @Override
+    public String getValue() {
+        return value;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConnectDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConnectDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConnectDetails.java
new file mode 100644
index 0000000..6c30206
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeConnectDetails.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.action.details;
+
+import org.apache.nifi.action.Component;
+
+/**
+ *
+ */
+public class FlowChangeConnectDetails implements ConnectDetails {
+
+    private String sourceId;
+    private String sourceName;
+    private Component sourceType;
+    private String relationship;
+    private String destinationId;
+    private String destinationName;
+    private Component destinationType;
+
+    @Override
+    public String getSourceId() {
+        return sourceId;
+    }
+
+    public void setSourceId(String sourceId) {
+        this.sourceId = sourceId;
+    }
+
+    @Override
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public void setSourceName(String sourceName) {
+        this.sourceName = sourceName;
+    }
+
+    @Override
+    public Component getSourceType() {
+        return sourceType;
+    }
+
+    public void setSourceType(Component sourceType) {
+        this.sourceType = sourceType;
+    }
+
+    @Override
+    public String getDestinationId() {
+        return destinationId;
+    }
+
+    public void setDestinationId(String destinationId) {
+        this.destinationId = destinationId;
+    }
+
+    @Override
+    public String getDestinationName() {
+        return destinationName;
+    }
+
+    public void setDestinationName(String destinationName) {
+        this.destinationName = destinationName;
+    }
+
+    @Override
+    public Component getDestinationType() {
+        return destinationType;
+    }
+
+    public void setDestinationType(Component destinationType) {
+        this.destinationType = destinationType;
+    }
+
+    @Override
+    public String getRelationship() {
+        return relationship;
+    }
+
+    public void setRelationship(String relationship) {
+        this.relationship = relationship;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeMoveDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeMoveDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeMoveDetails.java
new file mode 100644
index 0000000..aca0e2b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangeMoveDetails.java
@@ -0,0 +1,65 @@
+/*
+ * 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.action.details;
+
+/**
+ *
+ */
+public class FlowChangeMoveDetails implements MoveDetails {
+
+    private String previousGroupId;
+    private String previousGroup;
+    private String groupId;
+    private String group;
+
+    @Override
+    public String getGroup() {
+        return group;
+    }
+
+    public void setGroup(String group) {
+        this.group = group;
+    }
+
+    @Override
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(String groupId) {
+        this.groupId = groupId;
+    }
+
+    @Override
+    public String getPreviousGroup() {
+        return previousGroup;
+    }
+
+    public void setPreviousGroup(String previousGroup) {
+        this.previousGroup = previousGroup;
+    }
+
+    @Override
+    public String getPreviousGroupId() {
+        return previousGroupId;
+    }
+
+    public void setPreviousGroupId(String previousGroupId) {
+        this.previousGroupId = previousGroupId;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangePurgeDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangePurgeDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangePurgeDetails.java
new file mode 100644
index 0000000..bfc1d4d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/FlowChangePurgeDetails.java
@@ -0,0 +1,46 @@
+/*
+ * 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.action.details;
+
+import java.util.Date;
+
+/**
+ *
+ */
+public class FlowChangePurgeDetails implements PurgeDetails {
+
+    private Date endDate;
+
+    /**
+     * The end date for this purge action.
+     *
+     * @return date at which the purge ends
+     */
+    @Override
+    public Date getEndDate() {
+        return endDate;
+    }
+
+    /**
+     * Establishes the end data for this purge action
+     * @param endDate date at which the purge ends
+     */
+    public void setEndDate(Date endDate) {
+        this.endDate = endDate;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/MoveDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/MoveDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/MoveDetails.java
deleted file mode 100644
index 7585127..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/MoveDetails.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.action.details;
-
-/**
- *
- */
-public class MoveDetails extends ActionDetails {
-
-    private String previousGroupId;
-    private String previousGroup;
-    private String groupId;
-    private String group;
-
-    public String getGroup() {
-        return group;
-    }
-
-    public void setGroup(String group) {
-        this.group = group;
-    }
-
-    public String getGroupId() {
-        return groupId;
-    }
-
-    public void setGroupId(String groupId) {
-        this.groupId = groupId;
-    }
-
-    public String getPreviousGroup() {
-        return previousGroup;
-    }
-
-    public void setPreviousGroup(String previousGroup) {
-        this.previousGroup = previousGroup;
-    }
-
-    public String getPreviousGroupId() {
-        return previousGroupId;
-    }
-
-    public void setPreviousGroupId(String previousGroupId) {
-        this.previousGroupId = previousGroupId;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/PurgeDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
deleted file mode 100644
index 5b54bb8..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/details/PurgeDetails.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.action.details;
-
-import java.util.Date;
-
-/**
- *
- */
-public class PurgeDetails extends ActionDetails {
-
-    private Date endDate;
-
-    /**
-     * The end date for this purge action.
-     *
-     * @return date at which the purge ends
-     */
-    public Date getEndDate() {
-        return endDate;
-    }
-
-    /**
-     * Establishes the end data for this purge action
-     * @param endDate date at which the purge ends
-     */
-    public void setEndDate(Date endDate) {
-        this.endDate = endDate;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerAuditor.java
index f1e31dd..cede675 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerAuditor.java
@@ -21,8 +21,9 @@ import java.util.Collection;
 import java.util.Date;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.web.controller.ControllerFacade;
@@ -70,14 +71,14 @@ public class ControllerAuditor extends NiFiAuditor {
                 Collection<Action> actions = new ArrayList<>();
 
                 // create the configuration details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Controller Name");
                 configDetails.setValue(name);
                 configDetails.setPreviousValue(previousName);
 
                 // create the config action
-                Action configAction = new Action();
-                configAction.setUserDn(user.getDn());
+                FlowChangeAction configAction = new FlowChangeAction();
+                configAction.setUserIdentity(user.getDn());
                 configAction.setUserName(user.getUserName());
                 configAction.setOperation(Operation.Configure);
                 configAction.setTimestamp(new Date());
@@ -123,14 +124,14 @@ public class ControllerAuditor extends NiFiAuditor {
                 Collection<Action> actions = new ArrayList<>();
 
                 // create the configuration details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Controller Comments");
                 configDetails.setValue(comments);
                 configDetails.setPreviousValue(previousComments);
 
                 // create the config action
-                Action configAction = new Action();
-                configAction.setUserDn(user.getDn());
+                FlowChangeAction configAction = new FlowChangeAction();
+                configAction.setUserIdentity(user.getDn());
                 configAction.setUserName(user.getUserName());
                 configAction.setOperation(Operation.Configure);
                 configAction.setTimestamp(new Date());
@@ -176,14 +177,14 @@ public class ControllerAuditor extends NiFiAuditor {
                 Collection<Action> actions = new ArrayList<>();
 
                 // create the configure details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Controller Max Timer Driven Thread Count");
                 configDetails.setValue(String.valueOf(maxTimerDrivenThreadCount));
                 configDetails.setPreviousValue(String.valueOf(previousMaxTimerDrivenThreadCount));
 
                 // create the config action
-                Action configAction = new Action();
-                configAction.setUserDn(user.getDn());
+                FlowChangeAction configAction = new FlowChangeAction();
+                configAction.setUserIdentity(user.getDn());
                 configAction.setUserName(user.getUserName());
                 configAction.setOperation(Operation.Configure);
                 configAction.setTimestamp(new Date());
@@ -229,14 +230,14 @@ public class ControllerAuditor extends NiFiAuditor {
                 Collection<Action> actions = new ArrayList<>();
 
                 // create the configure details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Controller Max Event Driven Thread Count");
                 configDetails.setValue(String.valueOf(maxEventDrivenThreadCount));
                 configDetails.setPreviousValue(String.valueOf(previousMaxEventDrivenThreadCount));
 
                 // create the config action
-                Action configAction = new Action();
-                configAction.setUserDn(user.getDn());
+                FlowChangeAction configAction = new FlowChangeAction();
+                configAction.setUserIdentity(user.getDn());
                 configAction.setUserName(user.getUserName());
                 configAction.setOperation(Operation.Configure);
                 configAction.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
index 8ca3f0e..0187ee4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
@@ -25,10 +25,11 @@ import java.util.Set;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ProcessorNode;
@@ -121,7 +122,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
             Map<String, String> updatedValues = extractConfiguredPropertyValues(controllerService, controllerServiceDTO);
 
             // create the controller service details
-            ExtensionDetails serviceDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails serviceDetails = new FlowChangeExtensionDetails();
             serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName());
 
             // create a controller service action
@@ -159,14 +160,14 @@ public class ControllerServiceAuditor extends NiFiAuditor {
                         }
                     }
 
-                    final ConfigureDetails actionDetails = new ConfigureDetails();
+                    final FlowChangeConfigureDetails actionDetails = new FlowChangeConfigureDetails();
                     actionDetails.setName(property);
                     actionDetails.setValue(newValue);
                     actionDetails.setPreviousValue(oldValue);
 
                     // create a configuration action
-                    Action configurationAction = new Action();
-                    configurationAction.setUserDn(user.getDn());
+                    FlowChangeAction configurationAction = new FlowChangeAction();
+                    configurationAction.setUserIdentity(user.getDn());
                     configurationAction.setUserName(user.getUserName());
                     configurationAction.setOperation(operation);
                     configurationAction.setTimestamp(actionTimestamp);
@@ -185,8 +186,8 @@ public class ControllerServiceAuditor extends NiFiAuditor {
             // determine if the running state has changed and its not disabled
             if (isDisabled != updateIsDisabled) {
                 // create a controller service action
-                Action serviceAction = new Action();
-                serviceAction.setUserDn(user.getDn());
+                FlowChangeAction serviceAction = new FlowChangeAction();
+                serviceAction.setUserIdentity(user.getDn());
                 serviceAction.setUserName(user.getUserName());
                 serviceAction.setTimestamp(new Date());
                 serviceAction.setSourceId(controllerService.getIdentifier());
@@ -265,12 +266,12 @@ public class ControllerServiceAuditor extends NiFiAuditor {
                 final ProcessorNode processor = ((ProcessorNode) component);
 
                 // create the processor details
-                ExtensionDetails processorDetails = new ExtensionDetails();
+                FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
                 processorDetails.setType(processor.getProcessor().getClass().getSimpleName());
 
                 // create a processor action
-                Action processorAction = new Action();
-                processorAction.setUserDn(user.getDn());
+                FlowChangeAction processorAction = new FlowChangeAction();
+                processorAction.setUserIdentity(user.getDn());
                 processorAction.setUserName(user.getUserName());
                 processorAction.setTimestamp(new Date());
                 processorAction.setSourceId(processor.getIdentifier());
@@ -283,12 +284,12 @@ public class ControllerServiceAuditor extends NiFiAuditor {
                 final ReportingTaskNode reportingTask = ((ReportingTaskNode) component);
 
                 // create the reporting task details
-                ExtensionDetails processorDetails = new ExtensionDetails();
+                FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
                 processorDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName());
 
                 // create a reporting task action
-                Action reportingTaskAction = new Action();
-                reportingTaskAction.setUserDn(user.getDn());
+                FlowChangeAction reportingTaskAction = new FlowChangeAction();
+                reportingTaskAction.setUserIdentity(user.getDn());
                 reportingTaskAction.setUserName(user.getUserName());
                 reportingTaskAction.setTimestamp(new Date());
                 reportingTaskAction.setSourceId(reportingTask.getIdentifier());
@@ -301,12 +302,12 @@ public class ControllerServiceAuditor extends NiFiAuditor {
                 final ControllerServiceNode controllerService = ((ControllerServiceNode) component);
 
                 // create the controller service details
-                ExtensionDetails serviceDetails = new ExtensionDetails();
+                FlowChangeExtensionDetails serviceDetails = new FlowChangeExtensionDetails();
                 serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName());
 
                 // create a controller service action
-                Action serviceAction = new Action();
-                serviceAction.setUserDn(user.getDn());
+                FlowChangeAction serviceAction = new FlowChangeAction();
+                serviceAction.setUserIdentity(user.getDn());
                 serviceAction.setUserName(user.getUserName());
                 serviceAction.setTimestamp(new Date());
                 serviceAction.setSourceId(controllerService.getIdentifier());
@@ -373,7 +374,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
      * @return action
      */
     private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -381,12 +382,12 @@ public class ControllerServiceAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the controller service details
-            ExtensionDetails serviceDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails serviceDetails = new FlowChangeExtensionDetails();
             serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName());
 
             // create the controller service action for adding this controller service
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
index e864c89..51cb20c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java
@@ -20,6 +20,7 @@ import java.util.Date;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.ActionDetails;
 import org.apache.nifi.connectable.Funnel;
@@ -111,7 +112,7 @@ public class FunnelAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(Funnel funnel, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -119,8 +120,8 @@ public class FunnelAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the action for adding this funnel
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
index 4477cce..85f0b9f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java
@@ -19,6 +19,7 @@ package org.apache.nifi.audit;
 import java.util.ArrayList;
 import java.util.Collection;
 import org.apache.nifi.action.Action;
+import org.apache.nifi.action.details.FlowChangeMoveDetails;
 import org.apache.nifi.action.details.MoveDetails;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.cluster.context.ClusterContext;
@@ -62,22 +63,22 @@ public abstract class NiFiAuditor {
         // if we're a connected node, then put audit actions on threadlocal to propagate back to manager
         if (ctx != null) {
             ctx.getActions().addAll(actions);
-        } else {
-            // if we're the cluster manager, or a disconnected node, or running standalone, then audit actions
-            try {
-                // record the operations
-                auditService.addActions(actions);
-            } catch (Throwable t) {
-                logger.warn("Unable to record actions: " + t.getMessage());
-                if (logger.isDebugEnabled()) {
-                    logger.warn(StringUtils.EMPTY, t);
-                }
+        }
+
+        // always save the actions regardless of cluster or stand-alone
+        // all nodes in a cluster will have their own local copy without batching
+        try {
+            auditService.addActions(actions);
+        } catch (Throwable t) {
+            logger.warn("Unable to record actions: " + t.getMessage());
+            if (logger.isDebugEnabled()) {
+                logger.warn(StringUtils.EMPTY, t);
             }
         }
     }
 
     protected MoveDetails createMoveDetails(String previousGroupId, String newGroupId, Logger logger) {
-        MoveDetails moveDetails = null;
+        FlowChangeMoveDetails moveDetails = null;
 
         // get the groups in question
         ProcessGroup previousGroup = processGroupDAO.getProcessGroup(previousGroupId);
@@ -86,7 +87,7 @@ public abstract class NiFiAuditor {
         // ensure the groups were found
         if (previousGroup != null && newGroup != null) {
             // create the move details
-            moveDetails = new MoveDetails();
+            moveDetails = new FlowChangeMoveDetails();
             moveDetails.setPreviousGroupId(previousGroup.getIdentifier());
             moveDetails.setPreviousGroup(previousGroup.getName());
             moveDetails.setGroupId(newGroup.getIdentifier());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
index 2df883d..b07d64f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java
@@ -24,9 +24,10 @@ import java.util.Set;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.ScheduledState;
@@ -115,7 +116,7 @@ public class PortAuditor extends NiFiAuditor {
             // see if the name has changed
             if (name != null && portDTO.getName() != null && !name.equals(updatedPort.getName())) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Name");
                 configDetails.setValue(updatedPort.getName());
                 configDetails.setPreviousValue(name);
@@ -126,7 +127,7 @@ public class PortAuditor extends NiFiAuditor {
             // see if the comments has changed
             if (comments != null && portDTO.getComments() != null && !comments.equals(updatedPort.getComments())) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Comments");
                 configDetails.setValue(updatedPort.getComments());
                 configDetails.setPreviousValue(comments);
@@ -138,7 +139,7 @@ public class PortAuditor extends NiFiAuditor {
             if (isRootGroupPort) {
                 if (portDTO.getConcurrentlySchedulableTaskCount() != null && updatedPort.getMaxConcurrentTasks() != maxConcurrentTasks) {
                     // create the config details
-                    ConfigureDetails configDetails = new ConfigureDetails();
+                    FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                     configDetails.setName("Concurrent Tasks");
                     configDetails.setValue(String.valueOf(updatedPort.getMaxConcurrentTasks()));
                     configDetails.setPreviousValue(String.valueOf(maxConcurrentTasks));
@@ -157,7 +158,7 @@ public class PortAuditor extends NiFiAuditor {
                     // if users were added/removed
                     if (newUsers.size() > 0 || removedUsers.size() > 0) {
                         // create the config details
-                        ConfigureDetails configDetails = new ConfigureDetails();
+                        FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                         configDetails.setName("User Access Control");
                         configDetails.setValue(StringUtils.join(portDTO.getUserAccessControl(), ", "));
                         configDetails.setPreviousValue(StringUtils.join(existingUsers, ", "));
@@ -177,7 +178,7 @@ public class PortAuditor extends NiFiAuditor {
                     // if groups were added/removed
                     if (newGroups.size() > 0 || removedGroups.size() > 0) {
                         // create the config details
-                        ConfigureDetails configDetails = new ConfigureDetails();
+                        FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                         configDetails.setName("Group Access Control");
                         configDetails.setValue(StringUtils.join(portDTO.getGroupAccessControl(), ", "));
                         configDetails.setPreviousValue(StringUtils.join(existingGroups, ", "));
@@ -203,8 +204,8 @@ public class PortAuditor extends NiFiAuditor {
                 // create the actions
                 for (ActionDetails detail : configurationDetails) {
                     // create the port action for updating the name
-                    Action portAction = new Action();
-                    portAction.setUserDn(user.getDn());
+                    FlowChangeAction portAction = new FlowChangeAction();
+                    portAction.setUserIdentity(user.getDn());
                     portAction.setUserName(user.getUserName());
                     portAction.setOperation(Operation.Configure);
                     portAction.setTimestamp(timestamp);
@@ -223,8 +224,8 @@ public class PortAuditor extends NiFiAuditor {
             // determine if the running state has changed
             if (scheduledState != updatedScheduledState) {
                 // create a processor action
-                Action processorAction = new Action();
-                processorAction.setUserDn(user.getDn());
+                FlowChangeAction processorAction = new FlowChangeAction();
+                processorAction.setUserIdentity(user.getDn());
                 processorAction.setUserName(user.getUserName());
                 processorAction.setTimestamp(new Date());
                 processorAction.setSourceId(updatedPort.getIdentifier());
@@ -307,7 +308,7 @@ public class PortAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(Port port, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -321,8 +322,8 @@ public class PortAuditor extends NiFiAuditor {
             }
 
             // create the port action for adding this processor
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
index fa5ac70..d563555 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
@@ -21,10 +21,11 @@ import java.util.Collection;
 import java.util.Date;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
-import org.apache.nifi.action.details.MoveDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeMoveDetails;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
@@ -103,7 +104,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             // see if the name has changed
             if (name != null && updatedProcessGroup.getName() != null && !name.equals(updatedProcessGroup.getName())) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("name");
                 configDetails.setValue(updatedProcessGroup.getName());
                 configDetails.setPreviousValue(name);
@@ -114,7 +115,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             // see if the comments has changed
             if (comments != null && updatedProcessGroup.getComments() != null && !comments.equals(updatedProcessGroup.getComments())) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("comments");
                 configDetails.setValue(updatedProcessGroup.getComments());
                 configDetails.setPreviousValue(comments);
@@ -133,13 +134,13 @@ public class ProcessGroupAuditor extends NiFiAuditor {
                 for (ActionDetails detail : details) {
                     // determine the type of operation being performed
                     Operation operation = Operation.Configure;
-                    if (detail instanceof MoveDetails) {
+                    if (detail instanceof FlowChangeMoveDetails) {
                         operation = Operation.Move;
                     }
 
                     // create the port action for updating the name
-                    Action processGroupAction = new Action();
-                    processGroupAction.setUserDn(user.getDn());
+                    FlowChangeAction processGroupAction = new FlowChangeAction();
+                    processGroupAction.setUserIdentity(user.getDn());
                     processGroupAction.setUserName(user.getUserName());
                     processGroupAction.setOperation(operation);
                     processGroupAction.setTimestamp(timestamp);
@@ -155,8 +156,8 @@ public class ProcessGroupAuditor extends NiFiAuditor {
             // if the user was starting/stopping this process group
             if (processGroupDTO.isRunning() != null) {
                 // create a process group action
-                Action processGroupAction = new Action();
-                processGroupAction.setUserDn(user.getDn());
+                FlowChangeAction processGroupAction = new FlowChangeAction();
+                processGroupAction.setUserIdentity(user.getDn());
                 processGroupAction.setUserName(user.getUserName());
                 processGroupAction.setSourceId(processGroup.getIdentifier());
                 processGroupAction.setSourceName(processGroup.getName());
@@ -231,7 +232,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(ProcessGroup processGroup, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -240,8 +241,8 @@ public class ProcessGroupAuditor extends NiFiAuditor {
         if (user != null) {
 
             // create the process group action for adding this process group
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
index 23df486..b8a2c69 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java
@@ -29,10 +29,11 @@ import java.util.Set;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ScheduledState;
@@ -131,7 +132,7 @@ public class ProcessorAuditor extends NiFiAuditor {
             Map<String, String> updatedValues = extractConfiguredPropertyValues(processor, processorDTO);
 
             // create the processor details
-            ExtensionDetails processorDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
             processorDetails.setType(processor.getProcessor().getClass().getSimpleName());
 
             // create a processor action
@@ -169,14 +170,14 @@ public class ProcessorAuditor extends NiFiAuditor {
                         }
                     }
 
-                    final ConfigureDetails actionDetails = new ConfigureDetails();
+                    final FlowChangeConfigureDetails actionDetails = new FlowChangeConfigureDetails();
                     actionDetails.setName(property);
                     actionDetails.setValue(newValue);
                     actionDetails.setPreviousValue(oldValue);
 
                     // create a configuration action
-                    Action configurationAction = new Action();
-                    configurationAction.setUserDn(user.getDn());
+                    FlowChangeAction configurationAction = new FlowChangeAction();
+                    configurationAction.setUserIdentity(user.getDn());
                     configurationAction.setUserName(user.getUserName());
                     configurationAction.setOperation(operation);
                     configurationAction.setTimestamp(actionTimestamp);
@@ -195,8 +196,8 @@ public class ProcessorAuditor extends NiFiAuditor {
             // determine if the running state has changed and its not disabled
             if (scheduledState != updatedScheduledState) {
                 // create a processor action
-                Action processorAction = new Action();
-                processorAction.setUserDn(user.getDn());
+                FlowChangeAction processorAction = new FlowChangeAction();
+                processorAction.setUserIdentity(user.getDn());
                 processorAction.setUserName(user.getUserName());
                 processorAction.setTimestamp(new Date());
                 processorAction.setSourceId(processor.getIdentifier());
@@ -280,7 +281,7 @@ public class ProcessorAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(ProcessorNode processor, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -288,12 +289,12 @@ public class ProcessorAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the processor details
-            ExtensionDetails processorDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
             processorDetails.setType(processor.getProcessor().getClass().getSimpleName());
 
             // create the processor action for adding this processor
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
index c55a1c6..f90d572 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java
@@ -26,10 +26,12 @@ import java.util.Map;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
 import org.apache.nifi.action.details.ConnectDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConnectDetails;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
@@ -179,14 +181,14 @@ public class RelationshipAuditor extends NiFiAuditor {
                 // ensure the value is changing
                 if (oldValue == null || newValue == null || !newValue.equals(oldValue)) {
                     // create the config details
-                    ConfigureDetails configurationDetails = new ConfigureDetails();
+                    FlowChangeConfigureDetails configurationDetails = new FlowChangeConfigureDetails();
                     configurationDetails.setName(property);
                     configurationDetails.setValue(newValue);
                     configurationDetails.setPreviousValue(oldValue);
 
                     // create a configuration action
-                    Action configurationAction = new Action();
-                    configurationAction.setUserDn(user.getDn());
+                    FlowChangeAction configurationAction = new FlowChangeAction();
+                    configurationAction.setUserIdentity(user.getDn());
                     configurationAction.setUserName(user.getUserName());
                     configurationAction.setOperation(Operation.Configure);
                     configurationAction.setTimestamp(actionTimestamp);
@@ -263,7 +265,7 @@ public class RelationshipAuditor extends NiFiAuditor {
         final String formattedRelationships = relationshipNames.isEmpty() ? StringUtils.EMPTY : StringUtils.join(relationshipNames, ", ");
 
         // create the connect details
-        final ConnectDetails connectDetails = new ConnectDetails();
+        final FlowChangeConnectDetails connectDetails = new FlowChangeConnectDetails();
         connectDetails.setSourceId(source.getIdentifier());
         connectDetails.setSourceName(source.getName());
         connectDetails.setSourceType(sourceType);
@@ -327,7 +329,7 @@ public class RelationshipAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecordForConnection(Connection connection, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -350,8 +352,8 @@ public class RelationshipAuditor extends NiFiAuditor {
             Date actionTimestamp = new Date();
 
             // create a new relationship action
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(actionTimestamp);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
index f201fe2..e145a62 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java
@@ -23,10 +23,11 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.RemoteProcessGroupDetails;
+import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
@@ -123,7 +124,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
             // see if the communications timeout has changed
             if (remoteProcessGroupDTO.getCommunicationsTimeout() != null && !updatedRemoteProcessGroup.getCommunicationsTimeout().equals(communicationsTimeout)) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Communications Timeout");
                 configDetails.setValue(updatedRemoteProcessGroup.getCommunicationsTimeout());
                 configDetails.setPreviousValue(communicationsTimeout);
@@ -134,7 +135,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
             // see if the yield duration has changed
             if (remoteProcessGroupDTO.getYieldDuration() != null && !updatedRemoteProcessGroup.getYieldDuration().equals(yieldDuration)) {
                 // create the config details
-                ConfigureDetails configDetails = new ConfigureDetails();
+                FlowChangeConfigureDetails configDetails = new FlowChangeConfigureDetails();
                 configDetails.setName("Yield Duration");
                 configDetails.setValue(updatedRemoteProcessGroup.getYieldDuration());
                 configDetails.setPreviousValue(yieldDuration);
@@ -162,7 +163,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
                             final Integer previousConcurrentTasks = concurrentTasks.get(remotePortDTO.getId());
                             if (previousConcurrentTasks != null && remotePort.getMaxConcurrentTasks() != previousConcurrentTasks) {
                                 // create the config details
-                                ConfigureDetails concurrentTasksDetails = new ConfigureDetails();
+                                FlowChangeConfigureDetails concurrentTasksDetails = new FlowChangeConfigureDetails();
                                 concurrentTasksDetails.setName("Concurrent Tasks");
                                 concurrentTasksDetails.setValue(String.valueOf(remotePort.getMaxConcurrentTasks()));
                                 concurrentTasksDetails.setPreviousValue(String.valueOf(previousConcurrentTasks));
@@ -177,7 +178,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
                             final Boolean previousCompression = compression.get(remotePortDTO.getId());
                             if (previousCompression != null && remotePort.isUseCompression() != previousCompression) {
                                 // create the config details
-                                ConfigureDetails compressionDetails = new ConfigureDetails();
+                                FlowChangeConfigureDetails compressionDetails = new FlowChangeConfigureDetails();
                                 compressionDetails.setName("Compressed");
                                 compressionDetails.setValue(String.valueOf(remotePort.isUseCompression()));
                                 compressionDetails.setPreviousValue(String.valueOf(previousCompression));
@@ -204,7 +205,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
                             final Integer previousConcurrentTasks = concurrentTasks.get(remotePortDTO.getId());
                             if (previousConcurrentTasks != null && remotePort.getMaxConcurrentTasks() != previousConcurrentTasks) {
                                 // create the config details
-                                ConfigureDetails concurrentTasksDetails = new ConfigureDetails();
+                                FlowChangeConfigureDetails concurrentTasksDetails = new FlowChangeConfigureDetails();
                                 concurrentTasksDetails.setName("Concurrent Tasks");
                                 concurrentTasksDetails.setValue(String.valueOf(remotePort.getMaxConcurrentTasks()));
                                 concurrentTasksDetails.setPreviousValue(String.valueOf(previousConcurrentTasks));
@@ -219,7 +220,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
                             final Boolean previousCompression = compression.get(remotePortDTO.getId());
                             if (previousCompression != null && remotePort.isUseCompression() != previousCompression) {
                                 // create the config details
-                                ConfigureDetails compressionDetails = new ConfigureDetails();
+                                FlowChangeConfigureDetails compressionDetails = new FlowChangeConfigureDetails();
                                 compressionDetails.setName("Compressed");
                                 compressionDetails.setValue(String.valueOf(remotePort.isUseCompression()));
                                 compressionDetails.setPreviousValue(String.valueOf(previousCompression));
@@ -234,7 +235,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
             Collection<Action> actions = new ArrayList<>();
 
             // create the remote process group details
-            RemoteProcessGroupDetails remoteProcessGroupDetails = new RemoteProcessGroupDetails();
+            FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails();
             remoteProcessGroupDetails.setUri(remoteProcessGroup.getTargetUri().toString());
 
             // save the actions if necessary
@@ -244,8 +245,8 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
                 // create the actions
                 for (ActionDetails detail : details) {
                     // create the port action for updating the name
-                    Action remoteProcessGroupAction = new Action();
-                    remoteProcessGroupAction.setUserDn(user.getDn());
+                    FlowChangeAction remoteProcessGroupAction = new FlowChangeAction();
+                    remoteProcessGroupAction.setUserIdentity(user.getDn());
                     remoteProcessGroupAction.setUserName(user.getUserName());
                     remoteProcessGroupAction.setOperation(Operation.Configure);
                     remoteProcessGroupAction.setTimestamp(timestamp);
@@ -265,8 +266,8 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
             // determine if the running state has changed
             if (transmissionState != updatedTransmissionState) {
                 // create a processor action
-                Action remoteProcessGroupAction = new Action();
-                remoteProcessGroupAction.setUserDn(user.getDn());
+                FlowChangeAction remoteProcessGroupAction = new FlowChangeAction();
+                remoteProcessGroupAction.setUserIdentity(user.getDn());
                 remoteProcessGroupAction.setUserName(user.getUserName());
                 remoteProcessGroupAction.setTimestamp(new Date());
                 remoteProcessGroupAction.setSourceId(updatedRemoteProcessGroup.getIdentifier());
@@ -342,7 +343,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(RemoteProcessGroup remoteProcessGroup, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -350,12 +351,12 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the remote process group details
-            RemoteProcessGroupDetails remoteProcessGroupDetails = new RemoteProcessGroupDetails();
+            FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails();
             remoteProcessGroupDetails.setUri(remoteProcessGroup.getTargetUri().toString());
 
             // create the remote process group action
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
index bad91ec..712f99a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
@@ -25,10 +25,11 @@ import java.util.Set;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
 import org.apache.nifi.action.details.ActionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
@@ -114,7 +115,7 @@ public class ReportingTaskAuditor extends NiFiAuditor {
             Map<String, String> updatedValues = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO);
 
             // create the reporting task details
-            ExtensionDetails taskDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails taskDetails = new FlowChangeExtensionDetails();
             taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName());
 
             // create a reporting task action
@@ -152,14 +153,14 @@ public class ReportingTaskAuditor extends NiFiAuditor {
                         }
                     }
 
-                    final ConfigureDetails actionDetails = new ConfigureDetails();
+                    final FlowChangeConfigureDetails actionDetails = new FlowChangeConfigureDetails();
                     actionDetails.setName(property);
                     actionDetails.setValue(newValue);
                     actionDetails.setPreviousValue(oldValue);
 
                     // create a configuration action
-                    Action configurationAction = new Action();
-                    configurationAction.setUserDn(user.getDn());
+                    FlowChangeAction configurationAction = new FlowChangeAction();
+                    configurationAction.setUserIdentity(user.getDn());
                     configurationAction.setUserName(user.getUserName());
                     configurationAction.setOperation(operation);
                     configurationAction.setTimestamp(actionTimestamp);
@@ -178,8 +179,8 @@ public class ReportingTaskAuditor extends NiFiAuditor {
             // determine if the running state has changed and its not disabled
             if (scheduledState != updatedScheduledState) {
                 // create a reporting task action
-                Action taskAction = new Action();
-                taskAction.setUserDn(user.getDn());
+                FlowChangeAction taskAction = new FlowChangeAction();
+                taskAction.setUserIdentity(user.getDn());
                 taskAction.setUserName(user.getUserName());
                 taskAction.setTimestamp(new Date());
                 taskAction.setSourceId(reportingTask.getIdentifier());
@@ -262,7 +263,7 @@ public class ReportingTaskAuditor extends NiFiAuditor {
      * @return action
      */
     public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation, ActionDetails actionDetails) {
-        Action action = null;
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -270,12 +271,12 @@ public class ReportingTaskAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the reporting task details
-            ExtensionDetails taskDetails = new ExtensionDetails();
+            FlowChangeExtensionDetails taskDetails = new FlowChangeExtensionDetails();
             taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName());
 
             // create the reporting task action for adding this reporting task
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(new Date());


[44/50] [abbrv] nifi git commit: NIFI-883 Fixing issue HandleHttpRequest had with PrimaryNodeOnly scheduling

Posted by ma...@apache.org.
NIFI-883 Fixing issue HandleHttpRequest had with PrimaryNodeOnly scheduling

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: 2ae49026e898ef9ea154990f4d3a5da3ee8d8129
Parents: 37e2f17
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Mon Nov 2 10:17:32 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 10:32:14 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/HandleHttpRequest.java  | 23 +++++++++++++++++++-
 1 file changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/2ae49026/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
index 49bad40..1be8dd9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java
@@ -34,6 +34,7 @@ import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
 import javax.security.cert.X509Certificate;
@@ -226,6 +227,7 @@ public class HandleHttpRequest extends AbstractProcessor {
             .build();
 
     private volatile Server server;
+    private AtomicBoolean initialized = new AtomicBoolean(false);
     private final BlockingQueue<HttpRequestContainer> containerQueue = new LinkedBlockingQueue<>(50);
 
     @Override
@@ -255,7 +257,15 @@ public class HandleHttpRequest extends AbstractProcessor {
     }
 
     @OnScheduled
-    public void initializeServer(final ProcessContext context) throws Exception {
+    public void clearInit(){
+        initialized.set(false);
+    }
+
+    private synchronized void initializeServer(final ProcessContext context) throws Exception {
+        if(initialized.get()){
+            return;
+        }
+
         final String host = context.getProperty(HOSTNAME).getValue();
         final int port = context.getProperty(PORT).asInteger();
         final SSLContextService sslService = context.getProperty(SSL_CONTEXT).asControllerService(SSLContextService.class);
@@ -402,6 +412,8 @@ public class HandleHttpRequest extends AbstractProcessor {
         server.start();
 
         getLogger().info("Server started and listening on port " + getPort());
+
+        initialized.set(true);
     }
 
     protected int getPort() {
@@ -452,6 +464,15 @@ public class HandleHttpRequest extends AbstractProcessor {
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        try {
+            if(!initialized.get()) {
+                initializeServer(context);
+            }
+        } catch (Exception e) {
+            context.yield();
+            throw new ProcessException("Failed to initialize the server",e);
+        }
+
         final HttpRequestContainer container = containerQueue.poll();
         if (container == null) {
             return;


[50/50] [abbrv] nifi git commit: NIFI-730: Completed merging of master

Posted by ma...@apache.org.
NIFI-730: Completed merging of master


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

Branch: refs/heads/NIFI-730
Commit: dbf0c7893fef964bfbb3a4c039c756396587ce12
Parents: e4cebba
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Nov 2 14:31:28 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Nov 2 14:31:28 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/controller/FlowController.java  | 73 +++++++++++---------
 1 file changed, 40 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/dbf0c789/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 701459c..a3246fd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -16,39 +16,11 @@
  */
 package org.apache.nifi.controller;
 
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import javax.net.ssl.SSLContext;
-
+import com.sun.jersey.api.client.ClientHandlerException;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.action.Action;
+import org.apache.nifi.admin.service.AuditService;
+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;
@@ -141,6 +113,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
 import org.apache.nifi.groups.StandardProcessGroup;
+import org.apache.nifi.history.History;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.logging.ControllerServiceLogObserver;
 import org.apache.nifi.logging.LogLevel;
@@ -203,7 +176,36 @@ import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.sun.jersey.api.client.ClientHandlerException;
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static java.util.Objects.requireNonNull;
 
 public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, Heartbeater, QueueProvider {
 
@@ -345,11 +347,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
         final UserService userService,
+        final AuditService auditService,
         final StringEncryptor encryptor) {
         return new FlowController(
             flowFileEventRepo,
             properties,
             userService,
+            auditService,
             encryptor,
             /* configuredForClustering */ false,
             /* NodeProtocolSender */ null);
@@ -359,12 +363,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
         final UserService userService,
+        final AuditService auditService,
         final StringEncryptor encryptor,
         final NodeProtocolSender protocolSender) {
         final FlowController flowController = new FlowController(
             flowFileEventRepo,
             properties,
             userService,
+            auditService,
             encryptor,
             /* configuredForClustering */ true,
             /* NodeProtocolSender */ protocolSender);
@@ -378,6 +384,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final FlowFileEventRepository flowFileEventRepo,
         final NiFiProperties properties,
         final UserService userService,
+        final AuditService auditService,
         final StringEncryptor encryptor,
         final boolean configuredForClustering,
         final NodeProtocolSender protocolSender) {


[17/50] [abbrv] nifi git commit: NIFI-516 adding option to StandardProcessSession.read to close stream

Posted by ma...@apache.org.
NIFI-516 adding option to StandardProcessSession.read to close stream

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: b885f955f4ee97caeaf5c3a28aab967db1be4c94
Parents: 1c17386
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Wed Oct 7 10:50:07 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Oct 26 20:23:13 2015 -0400

----------------------------------------------------------------------
 .../apache/nifi/processor/ProcessSession.java   | 27 ++++++++++++++
 .../apache/nifi/util/MockProcessSession.java    |  8 ++++
 .../repository/BatchingSessionFactory.java      |  5 +++
 .../repository/StandardProcessSession.java      | 18 +++++++--
 .../repository/TestStandardProcessSession.java  | 39 +++++++++++++++++++-
 .../nifi/processors/standard/MergeContent.java  |  6 +--
 6 files changed, 95 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
index ed46d68..e1e98d5 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
@@ -509,6 +509,33 @@ public interface ProcessSession {
     void read(FlowFile source, InputStreamCallback reader) throws FlowFileAccessException;
 
     /**
+     * Executes the given callback against the contents corresponding to the
+     * given FlowFile.
+     *
+     * <i>Note</i>: The OutputStream provided to the given OutputStreamCallback
+     * will not be accessible once this method has completed its execution.
+     *
+     * @param source flowfile to retrieve content of
+     * @param allowSessionStreamManagement allow session to hold the stream open for performance reasons
+     * @param reader that will be called to read the flowfile content
+     * @throws IllegalStateException if detected that this method is being
+     * called from within a callback of another method in this session and for
+     * the given FlowFile(s)
+     * @throws FlowFileHandlingException if the given FlowFile is already
+     * transferred or removed or doesn't belong to this session. Automatic
+     * rollback will occur.
+     * @throws MissingFlowFileException if the given FlowFile content cannot be
+     * found. The FlowFile should no longer be reference, will be internally
+     * destroyed, and the session is automatically rolled back and what is left
+     * of the FlowFile is destroyed.
+     * @throws FlowFileAccessException if some IO problem occurs accessing
+     * FlowFile content; if an attempt is made to access the InputStream
+     * provided to the given InputStreamCallback after this method completed its
+     * execution
+     */
+    void read(FlowFile source, boolean allowSessionStreamManagement, InputStreamCallback reader) throws FlowFileAccessException;
+
+    /**
      * Combines the content of all given source FlowFiles into a single given
      * destination FlowFile.
      *

http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index 1060854..2045acd 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@ -400,6 +400,11 @@ public class MockProcessSession implements ProcessSession {
 
     @Override
     public void read(final FlowFile flowFile, final InputStreamCallback callback) {
+        read(flowFile, false, callback);
+    }
+
+    @Override
+    public void read(final FlowFile flowFile, boolean allowSessionStreamManagement, final InputStreamCallback callback) {
         if (callback == null || flowFile == null) {
             throw new IllegalArgumentException("argument cannot be null");
         }
@@ -413,6 +418,9 @@ public class MockProcessSession implements ProcessSession {
         final ByteArrayInputStream bais = new ByteArrayInputStream(mock.getData());
         try {
             callback.process(bais);
+            if(!allowSessionStreamManagement){
+                bais.close();
+            }
         } catch (final IOException e) {
             throw new ProcessException(e.toString(), e);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index d5dba82..083510d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@ -188,6 +188,11 @@ public class BatchingSessionFactory implements ProcessSessionFactory {
         }
 
         @Override
+        public void read(FlowFile source, boolean allowSessionStreamManagement, InputStreamCallback reader) {
+            session.read(source, allowSessionStreamManagement, reader);
+        }
+
+        @Override
         public FlowFile merge(Collection<FlowFile> sources, FlowFile destination) {
             return session.merge(sources, destination);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 3ba7e4e..98a0b87 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -1770,6 +1770,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
     @Override
     public void read(final FlowFile source, final InputStreamCallback reader) {
+        read(source, false, reader);
+    }
+
+    @Override
+    public void read(FlowFile source, boolean allowSessionStreamManagement, InputStreamCallback reader) {
         validateRecordState(source);
         final StandardRepositoryRecord record = records.get(source);
 
@@ -1780,9 +1785,9 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         }
 
         try (final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset());
-                final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
-                final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn);
-                final ByteCountingInputStream countingStream = new ByteCountingInputStream(disableOnCloseIn, this.bytesRead)) {
+            final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
+            final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn);
+            final ByteCountingInputStream countingStream = new ByteCountingInputStream(disableOnCloseIn, this.bytesRead)) {
 
             // We want to differentiate between IOExceptions thrown by the repository and IOExceptions thrown from
             // Processor code. As a result, as have the FlowFileAccessInputStream that catches IOException from the repository
@@ -1795,6 +1800,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             try {
                 recursionSet.add(source);
                 reader.process(ffais);
+
+                // Allow processors to close the file after reading to avoid too many files open or do smart session stream management.
+                if(!allowSessionStreamManagement){
+                    currentReadClaimStream.close();
+                    currentReadClaimStream = null;
+                }
             } catch (final ContentNotFoundException cnfe) {
                 cnfeThrown = true;
                 throw cnfe;
@@ -1806,6 +1817,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
                     throw ffais.getContentNotFoundException();
                 }
             }
+
         } catch (final ContentNotFoundException nfe) {
             handleContentNotFound(nfe, record);
         } catch (final IOException ex) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 0e11923..743e185 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -85,6 +85,7 @@ public class TestStandardProcessSession {
     private StandardProcessSession session;
     private MockContentRepository contentRepo;
     private FlowFileQueue flowFileQueue;
+    private ProcessContext context;
 
     private ProvenanceEventRepository provenanceRepo;
     private MockFlowFileRepository flowFileRepo;
@@ -187,7 +188,7 @@ public class TestStandardProcessSession {
         contentRepo.initialize(new StandardResourceClaimManager());
         flowFileRepo = new MockFlowFileRepository();
 
-        final ProcessContext context = new ProcessContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo);
+        context = new ProcessContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo);
         session = new StandardProcessSession(context);
     }
 
@@ -329,7 +330,7 @@ public class TestStandardProcessSession {
         final FlowFile flowFile = session.get();
         assertNotNull(flowFile);
         final ObjectHolder<InputStream> inputStreamHolder = new ObjectHolder<>(null);
-        session.read(flowFile, new InputStreamCallback() {
+        session.read(flowFile, true , new InputStreamCallback() {
             @Override
             public void process(final InputStream inputStream) throws IOException {
                 inputStreamHolder.set(inputStream);
@@ -721,6 +722,40 @@ public class TestStandardProcessSession {
     }
 
     @Test
+    public void testManyFilesOpened() throws IOException {
+
+        StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000];
+        for(int i = 0; i<70000;i++){
+            standardProcessSessions[i] = new StandardProcessSession(context);
+
+            FlowFile flowFile = standardProcessSessions[i].create();
+            final byte[] buff = new byte["Hello".getBytes().length];
+
+            flowFile = standardProcessSessions[i].append(flowFile, new OutputStreamCallback() {
+                @Override
+                public void process(OutputStream out) throws IOException {
+                    out.write("Hello".getBytes());
+                }
+            });
+
+            try {
+                standardProcessSessions[i].read(flowFile, false, new InputStreamCallback() {
+                    @Override
+                    public void process(final InputStream in) throws IOException {
+                        StreamUtils.fillBuffer(in, buff);
+                    }
+                });
+            } catch (Exception e){
+                System.out.println("Failed at file:"+i);
+                throw e;
+            }
+            if(i%1000==0){
+                System.out.println("i:"+i);
+            }
+        }
+    }
+
+    @Test
     public void testMissingFlowFileExceptionThrownWhenUnableToReadDataStreamCallback() {
         final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
             .addAttribute("uuid", "12345678-1234-1234-1234-123456789012")

http://git-wip-us.apache.org/repos/asf/nifi/blob/b885f955/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
index 2cad11e..afc2705 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java
@@ -567,7 +567,7 @@ public class MergeContent extends BinFiles {
                     final Iterator<FlowFileSessionWrapper> itr = wrappers.iterator();
                     while (itr.hasNext()) {
                         final FlowFileSessionWrapper wrapper = itr.next();
-                        wrapper.getSession().read(wrapper.getFlowFile(), new InputStreamCallback() {
+                        wrapper.getSession().read(wrapper.getFlowFile(), false, new InputStreamCallback() {
                             @Override
                             public void process(final InputStream in) throws IOException {
                                 StreamUtils.copy(in, out);
@@ -780,7 +780,7 @@ public class MergeContent extends BinFiles {
 
                         for (final FlowFileSessionWrapper wrapper : wrappers) {
                             final FlowFile flowFile = wrapper.getFlowFile();
-                            wrapper.getSession().read(flowFile, new InputStreamCallback() {
+                            wrapper.getSession().read(flowFile, false, new InputStreamCallback() {
                                 @Override
                                 public void process(final InputStream rawIn) throws IOException {
                                     try (final InputStream in = new BufferedInputStream(rawIn)) {
@@ -893,7 +893,7 @@ public class MergeContent extends BinFiles {
                     try (final OutputStream out = new BufferedOutputStream(rawOut)) {
                         for (final FlowFileSessionWrapper wrapper : wrappers) {
                             final FlowFile flowFile = wrapper.getFlowFile();
-                            wrapper.getSession().read(flowFile, new InputStreamCallback() {
+                            wrapper.getSession().read(flowFile, false, new InputStreamCallback() {
                                 @Override
                                 public void process(InputStream in) throws IOException {
                                     boolean canMerge = true;


[11/50] [abbrv] nifi git commit: NIFI-820: - Creating a context menu item for accessing provenance directly for a selected component.

Posted by ma...@apache.org.
NIFI-820:
- Creating a context menu item for accessing provenance directly for a selected component.

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

Branch: refs/heads/NIFI-730
Commit: f8c3377c84189c4688abc7186a99fea44be501f0
Parents: 26edab3
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Oct 26 10:16:01 2015 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Oct 26 10:16:01 2015 -0400

----------------------------------------------------------------------
 .../partials/provenance/provenance-content.jsp  |  2 +
 .../nifi-web-ui/src/main/webapp/css/main.css    |  7 ++-
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 16 ++++++
 .../main/webapp/js/nf/canvas/nf-context-menu.js | 23 ++++++++-
 .../js/nf/provenance/nf-provenance-table.js     | 54 +++++++++++++-------
 .../webapp/js/nf/provenance/nf-provenance.js    | 16 +++++-
 6 files changed, 96 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/provenance/provenance-content.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/provenance/provenance-content.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/provenance/provenance-content.jsp
index 035e8d3..8e6a3db 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/provenance/provenance-content.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/provenance/provenance-content.jsp
@@ -15,7 +15,9 @@
   limitations under the License.
 --%>
 <%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
+<%@ taglib prefix="c" uri="http://java.sun.com/jsp/jstl/core" %>
 <div id="provenance">
+    <span id="intial-component-query" class="hidden"><c:out value="${param.componentId}"/></span>
     <span id="nifi-controller-uri" class="hidden"></span>
     <span id="nifi-content-viewer-url" class="hidden"></span>
     <div id="provenance-header-and-filter">

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
index aa12602..95ee641 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/main.css
@@ -57,10 +57,11 @@ div.context-menu-item.hover {
     color: #027fd2;
 }
 
-img.context-menu-item-img {
+.context-menu-item-img {
     float: left;
     width: 16px;
     height: 16px;
+    background-size: cover;
 }
 
 div.context-menu-item-text {
@@ -72,6 +73,10 @@ div.context-menu-item-text {
     font-size: 11px;
 }
 
+div.context-menu-provenance {
+    background-position: top left;
+}
+
 /*
     General Styles
 */

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
index c6ef75f..fbe5cec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-actions.js
@@ -427,6 +427,22 @@ nf.Actions = (function () {
         },
         
         /**
+         * Opens provenance with the component in the specified selection.
+         *
+         * @argument {selection} selection The selection
+         */
+        openProvenance: function (selection) {
+            if (selection.size() === 1) {
+                var selectionData = selection.datum();
+
+                // open the provenance page with the specified component
+                nf.Shell.showPage('provenance?' + $.param({
+                    componentId: selectionData.component.id
+                }));
+            }
+        },
+
+        /**
          * Starts the components in the specified selection.
          * 
          * @argument {selection} selection      The selection

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
index e652dd4..98f7a57 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
@@ -246,6 +246,20 @@ nf.ContextMenu = (function () {
     };
 
     /**
+     * Determines whether the current selection could have provenance.
+     *
+     * @param {selection} selection
+     */
+    var canAccessProvenance = function (selection) {
+        // ensure the correct number of components are selected
+        if (selection.size() !== 1) {
+            return false;
+        }
+
+        return !nf.CanvasUtils.isConnection(selection) && !nf.CanvasUtils.isProcessGroup(selection) && !nf.CanvasUtils.isRemoteProcessGroup(selection) && nf.Common.canAccessProvenance();
+    };
+
+    /**
      * Determines whether the current selection is a remote process group.
      * 
      * @param {selection} selection         
@@ -313,7 +327,12 @@ nf.ContextMenu = (function () {
                 $(this).removeClass('hover');
             }).appendTo(contextMenu);
 
-            $('<img class="context-menu-item-img"></img>').attr('src', item['img']).appendTo(menuItem);
+            // create the img and conditionally add the style
+            var img = $('<div class="context-menu-item-img"></div>').css('background-image', 'url(' + item['img'] + ')').appendTo(menuItem);
+            if (nf.Common.isDefinedAndNotNull(item['imgStyle'])) {
+                img.addClass(item['imgStyle']);
+            }
+            
             $('<div class="context-menu-item-text"></div>').text(item['text']).appendTo(menuItem);
             $('<div class="clear"></div>').appendTo(menuItem);
         }
@@ -360,6 +379,7 @@ nf.ContextMenu = (function () {
         {condition: canStartTransmission, menuItem: {img: 'images/iconTransmissionActive.png', text: 'Enable transmission', action: 'enableTransmission'}},
         {condition: canStopTransmission, menuItem: {img: 'images/iconTransmissionInactive.png', text: 'Disable transmission', action: 'disableTransmission'}},
         {condition: supportsStats, menuItem: {img: 'images/iconChart.png', text: 'Stats', action: 'showStats'}},
+        {condition: canAccessProvenance, menuItem: {img: 'images/iconProvenance.png', imgStyle: 'context-menu-provenance', text: 'Data provenance', action: 'openProvenance'}},
         {condition: canMoveToFront, menuItem: {img: 'images/iconToFront.png', text: 'Bring to front', action: 'toFront'}},
         {condition: isConnection, menuItem: {img: 'images/iconGoTo.png', text: 'Go to source', action: 'showSource'}},
         {condition: isConnection, menuItem: {img: 'images/iconGoTo.png', text: 'Go to destination', action: 'showDestination'}},
@@ -416,6 +436,7 @@ nf.ContextMenu = (function () {
 
                     addMenuItem(contextMenu, {
                         img: menuItem.img,
+                        imgStyle: menuItem.imgStyle, 
                         text: menuItem.text,
                         click: function (evt) {
                             executeAction(menuItem.action, selection, evt);

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
index 499925d..1e5550c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
@@ -258,19 +258,6 @@ nf.ProvenanceTable = (function () {
      * @param {boolean} isClustered     Whether or not this NiFi clustered
      */
     var initSearchDialog = function (isClustered) {
-        $.ajax({
-            type: 'GET',
-            url: config.urls.searchOptions,
-            dataType: 'json'
-        }).done(function (response) {
-            var provenanceOptions = response.provenanceOptions;
-
-            // load all searchable fields
-            $.each(provenanceOptions.searchableFields, function (_, field) {
-                appendSearchableField(field);
-            });
-        });
-
         // configure the start and end date picker
         $('#provenance-search-start-date, #provenance-search-end-date').datepicker({
             showAnim: '',
@@ -402,6 +389,19 @@ nf.ProvenanceTable = (function () {
                     }
                 }]
         });
+        
+        return $.ajax({
+            type: 'GET',
+            url: config.urls.searchOptions,
+            dataType: 'json'
+        }).done(function (response) {
+            var provenanceOptions = response.provenanceOptions;
+
+            // load all searchable fields
+            $.each(provenanceOptions.searchableFields, function (_, field) {
+                appendSearchableField(field);
+            });
+        });
     };
 
     /**
@@ -441,6 +441,11 @@ nf.ProvenanceTable = (function () {
         $('<div class="searchable-field-value"><input type="text" class="searchable-field-input"/></div>').appendTo(searchableField);
         $('<div class="clear"></div>').appendTo(searchableField);
 
+        // make the component id accessible for populating
+        if (field.id === 'ProcessorID') {
+            searchableField.find('input').addClass('searchable-component-id');
+        }
+
         // ensure the no searchable fields message is hidden
         $('#no-searchable-fields').hide();
     };
@@ -949,12 +954,23 @@ nf.ProvenanceTable = (function () {
          * @param {boolean} isClustered     Whether or not this instance is clustered
          */
         init: function (isClustered) {
-            return loadLineageCapabilities().done(function () {
-                initDetailsDialog();
-                initProvenanceQueryDialog();
-                initSearchDialog(isClustered);
-                initProvenanceTable(isClustered);
-            }).fail(nf.Common.handleAjaxError);
+            return $.Deferred(function (deferred) {
+                // handles init failure
+                var failure = function (xhr, status, error) {
+                    deferred.reject();
+                    nf.Common.handleAjaxError(xhr, status, error);
+                };
+                
+                // load the lineage capabilities
+                loadLineageCapabilities().done(function () {
+                    initDetailsDialog();
+                    initProvenanceQueryDialog();
+                    initProvenanceTable(isClustered);
+                    initSearchDialog(isClustered).done(function () {
+                        deferred.resolve();
+                    }).fail(failure);
+                }).fail(failure);
+            }).promise();
         },
         
         /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/f8c3377c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance.js
index df1a682..0edbe08 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance.js
@@ -178,8 +178,22 @@ nf.Provenance = (function () {
             $.when(loadControllerConfig(), loadAuthorities(), detectedCluster()).done(function () {
                 // create the provenance table
                 nf.ProvenanceTable.init(isClustered).done(function () {
+                    var search;
+                    
+                    // look for a processor id in the query search
+                    var initialComponentId = $('#intial-component-query').text();
+                    if ($.trim(initialComponentId) !== '') {
+                        // populate initial search component
+                        $('input.searchable-component-id').val(initialComponentId);
+                        
+                        // build the search criteria
+                        search = {
+                            'search[ProcessorID]': initialComponentId
+                        };
+                    }
+
                     // load the provenance table
-                    nf.ProvenanceTable.loadProvenanceTable();
+                    nf.ProvenanceTable.loadProvenanceTable(search);
 
                     // once the table is initialized, finish initializing the page
                     initializeProvenancePage().done(function () {


[25/50] [abbrv] nifi git commit: NIFI-986 Refactoring of action classes from nifi-user-actions to have interfaces in nifi-api, and adding getFlowChanges to EventAccess - Fixing empty java docs and adding sort by id asc to the history query - Changing use

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
index 20ce740..34382b3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java
@@ -24,10 +24,12 @@ import java.util.Set;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
-import org.apache.nifi.action.component.details.RemoteProcessGroupDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails;
 import org.apache.nifi.action.details.ConnectDetails;
+import org.apache.nifi.action.details.FlowChangeConnectDetails;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
@@ -136,10 +138,10 @@ public class SnippetAuditor extends NiFiAuditor {
 
         // remote processor groups
         for (final RemoteProcessGroupDTO remoteProcessGroup : snippet.getRemoteProcessGroups()) {
-            RemoteProcessGroupDetails remoteProcessGroupDetails = new RemoteProcessGroupDetails();
+            FlowChangeRemoteProcessGroupDetails remoteProcessGroupDetails = new FlowChangeRemoteProcessGroupDetails();
             remoteProcessGroupDetails.setUri(remoteProcessGroup.getTargetUri());
 
-            final Action action = generateAuditRecord(remoteProcessGroup.getId(), remoteProcessGroup.getName(), Component.RemoteProcessGroup, Operation.Add, timestamp);
+            final FlowChangeAction action = generateAuditRecord(remoteProcessGroup.getId(), remoteProcessGroup.getName(), Component.RemoteProcessGroup, Operation.Add, timestamp);
             action.setComponentDetails(remoteProcessGroupDetails);
             actions.add(action);
         }
@@ -151,10 +153,10 @@ public class SnippetAuditor extends NiFiAuditor {
 
         // processors
         for (final ProcessorDTO processor : snippet.getProcessors()) {
-            final ExtensionDetails processorDetails = new ExtensionDetails();
+            final FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
             processorDetails.setType(StringUtils.substringAfterLast(processor.getType(), "."));
 
-            final Action action = generateAuditRecord(processor.getId(), processor.getName(), Component.Processor, Operation.Add, timestamp);
+            final FlowChangeAction action = generateAuditRecord(processor.getId(), processor.getName(), Component.Processor, Operation.Add, timestamp);
             action.setComponentDetails(processorDetails);
             actions.add(action);
         }
@@ -174,7 +176,7 @@ public class SnippetAuditor extends NiFiAuditor {
             final String name = StringUtils.isBlank(connection.getName()) ? relationships : connection.getName();
 
             // create the connect details
-            ConnectDetails connectDetails = new ConnectDetails();
+            FlowChangeConnectDetails connectDetails = new FlowChangeConnectDetails();
             connectDetails.setSourceId(source.getId());
             connectDetails.setSourceName(source.getName());
             connectDetails.setSourceType(determineConnectableType(source));
@@ -184,7 +186,7 @@ public class SnippetAuditor extends NiFiAuditor {
             connectDetails.setDestinationType(determineConnectableType(destination));
 
             // create the audit record
-            final Action action = generateAuditRecord(connection.getId(), name, Component.Connection, Operation.Connect, timestamp);
+            final FlowChangeAction action = generateAuditRecord(connection.getId(), name, Component.Connection, Operation.Connect, timestamp);
             action.setActionDetails(connectDetails);
             actions.add(action);
         }
@@ -220,8 +222,8 @@ public class SnippetAuditor extends NiFiAuditor {
     /**
      * Generates an audit record for the creation of the specified funnel.
      */
-    private Action generateAuditRecord(String id, String name, Component type, Operation operation, Date timestamp) {
-        Action action = null;
+    private FlowChangeAction generateAuditRecord(String id, String name, Component type, Operation operation, Date timestamp) {
+        FlowChangeAction action = null;
 
         // get the current user
         NiFiUser user = NiFiUserUtils.getNiFiUser();
@@ -229,8 +231,8 @@ public class SnippetAuditor extends NiFiAuditor {
         // ensure the user was found
         if (user != null) {
             // create the action for adding this funnel
-            action = new Action();
-            action.setUserDn(user.getDn());
+            action = new FlowChangeAction();
+            action.setUserIdentity(user.getDn());
             action.setUserName(user.getUserName());
             action.setOperation(operation);
             action.setTimestamp(timestamp);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 2286213..39426c0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -37,8 +37,9 @@ import javax.ws.rs.WebApplicationException;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.details.PurgeDetails;
+import org.apache.nifi.action.details.FlowChangePurgeDetails;
 import org.apache.nifi.admin.service.AccountNotFoundException;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.admin.service.UserService;
@@ -1765,12 +1766,12 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         }
 
         // create the purge details
-        PurgeDetails details = new PurgeDetails();
+        FlowChangePurgeDetails details = new FlowChangePurgeDetails();
         details.setEndDate(endDate);
 
         // create a purge action to record that records are being removed
-        Action purgeAction = new Action();
-        purgeAction.setUserDn(user.getDn());
+        FlowChangeAction purgeAction = new FlowChangeAction();
+        purgeAction.setUserIdentity(user.getDn());
         purgeAction.setUserName(user.getUserName());
         purgeAction.setOperation(Operation.Purge);
         purgeAction.setTimestamp(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index 870ba56..8e56fc2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -34,9 +34,10 @@ import javax.ws.rs.core.Response;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@@ -121,21 +122,21 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
         final Date now = new Date();
         final Collection<Action> actions = new HashSet<>(configurationActions.size());
         for (final ConfigurationAction configurationAction : configurationActions) {
-            final ExtensionDetails extensionDetails = new ExtensionDetails();
+            final FlowChangeExtensionDetails extensionDetails = new FlowChangeExtensionDetails();
             extensionDetails.setType(configurationAction.getType());
 
-            final ConfigureDetails configureDetails = new ConfigureDetails();
+            final FlowChangeConfigureDetails configureDetails = new FlowChangeConfigureDetails();
             configureDetails.setName(configurationAction.getName());
             configureDetails.setPreviousValue(configurationAction.getPreviousValue());
             configureDetails.setValue(configurationAction.getValue());
 
-            final Action action = new Action();
+            final FlowChangeAction action = new FlowChangeAction();
             action.setTimestamp(now);
             action.setSourceId(configurationAction.getId());
             action.setSourceName(configurationAction.getName());
             action.setSourceType(componentType);
             action.setOperation(Operation.Configure);
-            action.setUserDn(getCurrentUserDn());
+            action.setUserIdentity(getCurrentUserDn());
             action.setUserName(getCurrentUserName());
             action.setComponentDetails(extensionDetails);
             action.setActionDetails(configureDetails);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
index 37ad804..03ce630 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java
@@ -34,9 +34,10 @@ import javax.ws.rs.core.Response;
 
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.Component;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
-import org.apache.nifi.action.component.details.ExtensionDetails;
-import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@@ -94,21 +95,21 @@ public class StandardNiFiWebContext implements NiFiWebContext {
         final Date now = new Date();
         final Collection<Action> actions = new HashSet<>(processorActions.size());
         for (final ProcessorConfigurationAction processorAction : processorActions) {
-            final ExtensionDetails processorDetails = new ExtensionDetails();
+            final FlowChangeExtensionDetails processorDetails = new FlowChangeExtensionDetails();
             processorDetails.setType(processorAction.getProcessorType());
 
-            final ConfigureDetails configureDetails = new ConfigureDetails();
+            final FlowChangeConfigureDetails configureDetails = new FlowChangeConfigureDetails();
             configureDetails.setName(processorAction.getName());
             configureDetails.setPreviousValue(processorAction.getPreviousValue());
             configureDetails.setValue(processorAction.getValue());
 
-            final Action action = new Action();
+            final FlowChangeAction action = new FlowChangeAction();
             action.setTimestamp(now);
             action.setSourceId(processorAction.getProcessorId());
             action.setSourceName(processorAction.getProcessorName());
             action.setSourceType(Component.Processor);
             action.setOperation(Operation.Configure);
-            action.setUserDn(getCurrentUserDn());
+            action.setUserIdentity(getCurrentUserDn());
             action.setUserName(getCurrentUserName());
             action.setComponentDetails(processorDetails);
             action.setActionDetails(configureDetails);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index aa51925..3bad5e3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -40,6 +40,7 @@ import javax.ws.rs.core.UriBuilder;
 import javax.ws.rs.core.UriBuilderException;
 import javax.ws.rs.core.UriInfo;
 import org.apache.nifi.action.Action;
+import org.apache.nifi.action.FlowChangeAction;
 import org.apache.nifi.action.Operation;
 import org.apache.nifi.cluster.context.ClusterContext;
 import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
@@ -203,12 +204,12 @@ public abstract class ApplicationResource {
                 clusterCtx.getActions().clear();
 
                 // create the batch action
-                Action batchAction = new Action();
+                FlowChangeAction batchAction = new FlowChangeAction();
                 batchAction.setOperation(Operation.Batch);
 
                 // copy values from prototype action
                 batchAction.setTimestamp(prototypeAction.getTimestamp());
-                batchAction.setUserDn(prototypeAction.getUserDn());
+                batchAction.setUserIdentity(prototypeAction.getUserIdentity());
                 batchAction.setUserName(prototypeAction.getUserName());
                 batchAction.setSourceId(prototypeAction.getSourceId());
                 batchAction.setSourceName(prototypeAction.getSourceName());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5cc2b04b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 16b114e..96a67f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -40,10 +40,16 @@ import javax.ws.rs.WebApplicationException;
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.component.details.ComponentDetails;
 import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeExtensionDetails;
+import org.apache.nifi.action.component.details.FlowChangeRemoteProcessGroupDetails;
 import org.apache.nifi.action.component.details.RemoteProcessGroupDetails;
 import org.apache.nifi.action.details.ActionDetails;
 import org.apache.nifi.action.details.ConfigureDetails;
 import org.apache.nifi.action.details.ConnectDetails;
+import org.apache.nifi.action.details.FlowChangeConfigureDetails;
+import org.apache.nifi.action.details.FlowChangeConnectDetails;
+import org.apache.nifi.action.details.FlowChangeMoveDetails;
+import org.apache.nifi.action.details.FlowChangePurgeDetails;
 import org.apache.nifi.action.details.MoveDetails;
 import org.apache.nifi.action.details.PurgeDetails;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -156,7 +162,7 @@ public final class DtoFactory {
         actionDto.setSourceName(action.getSourceName());
         actionDto.setSourceType(action.getSourceType().name());
         actionDto.setTimestamp(action.getTimestamp());
-        actionDto.setUserDn(action.getUserDn());
+        actionDto.setUserDn(action.getUserIdentity());
         actionDto.setUserName(action.getUserName());
         actionDto.setOperation(action.getOperation().name());
         actionDto.setActionDetails(createActionDetailsDto(action.getActionDetails()));
@@ -176,13 +182,13 @@ public final class DtoFactory {
             return null;
         }
 
-        if (actionDetails instanceof ConfigureDetails) {
+        if (actionDetails instanceof FlowChangeConfigureDetails) {
             final ConfigureDetailsDTO configureDetails = new ConfigureDetailsDTO();
             configureDetails.setName(((ConfigureDetails) actionDetails).getName());
             configureDetails.setPreviousValue(((ConfigureDetails) actionDetails).getPreviousValue());
             configureDetails.setValue(((ConfigureDetails) actionDetails).getValue());
             return configureDetails;
-        } else if (actionDetails instanceof ConnectDetails) {
+        } else if (actionDetails instanceof FlowChangeConnectDetails) {
             final ConnectDetailsDTO connectDetails = new ConnectDetailsDTO();
             connectDetails.setSourceId(((ConnectDetails) actionDetails).getSourceId());
             connectDetails.setSourceName(((ConnectDetails) actionDetails).getSourceName());
@@ -192,14 +198,14 @@ public final class DtoFactory {
             connectDetails.setDestinationName(((ConnectDetails) actionDetails).getDestinationName());
             connectDetails.setDestinationType(((ConnectDetails) actionDetails).getDestinationType().toString());
             return connectDetails;
-        } else if (actionDetails instanceof MoveDetails) {
+        } else if (actionDetails instanceof FlowChangeMoveDetails) {
             final MoveDetailsDTO moveDetails = new MoveDetailsDTO();
             moveDetails.setPreviousGroup(((MoveDetails) actionDetails).getPreviousGroup());
             moveDetails.setPreviousGroupId(((MoveDetails) actionDetails).getPreviousGroupId());
             moveDetails.setGroup(((MoveDetails) actionDetails).getGroup());
             moveDetails.setGroupId(((MoveDetails) actionDetails).getGroupId());
             return moveDetails;
-        } else if (actionDetails instanceof PurgeDetails) {
+        } else if (actionDetails instanceof FlowChangePurgeDetails) {
             final PurgeDetailsDTO purgeDetails = new PurgeDetailsDTO();
             purgeDetails.setEndDate(((PurgeDetails) actionDetails).getEndDate());
             return purgeDetails;
@@ -219,11 +225,11 @@ public final class DtoFactory {
             return null;
         }
 
-        if (componentDetails instanceof ExtensionDetails) {
+        if (componentDetails instanceof FlowChangeExtensionDetails) {
             final ExtensionDetailsDTO processorDetails = new ExtensionDetailsDTO();
             processorDetails.setType(((ExtensionDetails) componentDetails).getType());
             return processorDetails;
-        } else if (componentDetails instanceof RemoteProcessGroupDetails) {
+        } else if (componentDetails instanceof FlowChangeRemoteProcessGroupDetails) {
             final RemoteProcessGroupDetailsDTO remoteProcessGroupDetails = new RemoteProcessGroupDetailsDTO();
             remoteProcessGroupDetails.setUri(((RemoteProcessGroupDetails) componentDetails).getUri());
             return remoteProcessGroupDetails;


[36/50] [abbrv] nifi git commit: NIFI-869 Fixed SimpleProcessLogger to log correct messages Ensured that SimpleProcessLogger correctly interprets Throwable as discussed in JIRA Added tests

Posted by ma...@apache.org.
NIFI-869 Fixed SimpleProcessLogger to log correct messages Ensured that SimpleProcessLogger correctly interprets Throwable as discussed in JIRA Added tests

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/NIFI-730
Commit: a3d43d23dcf68310b518d35751d1835767d895a4
Parents: 90aea01
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Fri Oct 30 14:24:12 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Oct 30 15:52:33 2015 -0400

----------------------------------------------------------------------
 .../scheduling/StandardProcessScheduler.java    |   2 +-
 .../nifi/processor/SimpleProcessLogger.java     |  20 ++--
 .../nifi/processor/TestSimpleProcessLogger.java | 100 +++++++++++++++++++
 3 files changed, 113 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/a3d43d23/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 5d99d07..9ff58c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -655,7 +655,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                             final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
 
                             final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service);
-                            componentLog.error("failed to invoke @OnEnabled method due to {}", new Object[]{cause.toString()});
+                            componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
                             LOG.error("Failed to invoke @OnEnabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString());
                             if (LOG.isDebugEnabled()) {
                                 LOG.error("", cause);

http://git-wip-us.apache.org/repos/asf/nifi/blob/a3d43d23/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
index afd0c59..900ec77 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
@@ -62,8 +62,12 @@ public class SimpleProcessLogger implements ProcessorLog {
     }
 
     @Override
-    public void warn(final String msg, final Throwable t) {
-        warn("{} " + msg, new Object[]{component}, t);
+    public void warn(String msg, final Throwable t) {
+        //warn("{} " + msg, new Object[]{component}, t);
+        msg = "{} " + msg;
+        final Object[] os = {component, t.toString(), t};
+        logger.warn(msg, os);
+        logRepository.addLogMessage(LogLevel.WARN, msg, os, t);
     }
 
     @Override
@@ -101,8 +105,8 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void trace(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {component};
-        logger.trace(msg, os, t);
+        final Object[] os = {component, t.toString(), t};
+        logger.trace(msg, os);
         logRepository.addLogMessage(LogLevel.TRACE, msg, os, t);
     }
 
@@ -160,7 +164,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void info(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {component};
+        final Object[] os = {component, t.toString()};
 
         logger.info(msg, os);
         if (logger.isDebugEnabled()) {
@@ -207,12 +211,12 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void error(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {component};
-
-        logger.error(msg, os, t);
+        Object[] os = {component, t.toString()};
+        logger.error(msg, os);
         if (logger.isDebugEnabled()) {
             logger.error("", t);
         }
+
         logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/a3d43d23/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
new file mode 100644
index 0000000..2876abb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestSimpleProcessLogger.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.times;
+
+import java.lang.reflect.Field;
+
+import org.apache.nifi.reporting.ReportingTask;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.matchers.VarargMatcher;
+import org.slf4j.Logger;
+
+public class TestSimpleProcessLogger {
+	private final Exception e = new RuntimeException("intentional");
+
+	private  ReportingTask task;
+
+	private SimpleProcessLogger componentLog;
+
+	private Logger logger;
+
+	@Before
+	public void before(){
+		task = mock(ReportingTask.class);
+		when(task.getIdentifier()).thenReturn("foo");
+		when(task.toString()).thenReturn("MyTask");
+		componentLog = new SimpleProcessLogger(task.getIdentifier(), task);
+		try {
+			Field loggerField = componentLog.getClass().getDeclaredField("logger");
+			loggerField.setAccessible(true);
+			logger = mock(Logger.class);
+			loggerField.set(componentLog, logger);
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void validateDelegateLoggerReceivesThrowableToStringOnError() {
+		componentLog.error("Hello {}", e);
+		verify(logger, times(1)).error(anyString(), argThat(new MyVarargMatcher()));
+	}
+
+	@Test
+	public void validateDelegateLoggerReceivesThrowableToStringOnInfo() {
+		componentLog.info("Hello {}", e);
+		verify(logger, times(1)).info(anyString(), argThat(new MyVarargMatcher()));
+	}
+
+	@Test
+	public void validateDelegateLoggerReceivesThrowableToStringOnTrace() {
+		componentLog.trace("Hello {}", e);
+		verify(logger, times(1)).trace(anyString(), argThat(new MyVarargMatcher()));
+	}
+
+	@Test
+	public void validateDelegateLoggerReceivesThrowableToStringOnWarn() {
+		componentLog.warn("Hello {}", e);
+		verify(logger, times(1)).warn(anyString(), argThat(new MyVarargMatcher()));
+	}
+
+	/**
+	 *
+	 */
+	private class MyVarargMatcher extends ArgumentMatcher<Object[]> implements VarargMatcher {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public boolean matches(Object argument) {
+			Object[] args = (Object[]) argument;
+			assertEquals(task, args[0]);
+			assertEquals(e.toString(), args[1]);
+			return true;
+		}
+	}
+}


[29/50] [abbrv] nifi git commit: NIFI-1088: Ensure that FlowFile is penalized before routing to failure

Posted by ma...@apache.org.
NIFI-1088: Ensure that FlowFile is penalized before routing to failure


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

Branch: refs/heads/NIFI-730
Commit: 9515b7460713ba985a6d7c8ad033fe2c1ac98e3d
Parents: dc4004d
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Oct 30 14:25:27 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Oct 30 14:25:27 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/processors/kafka/PutKafka.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/9515b746/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
index cff285c..09025a4 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java
@@ -401,7 +401,7 @@ public class PutKafka extends AbstractProcessor {
                 getLogger().info("Successfully sent {} to Kafka in {} millis", new Object[] { flowFile, TimeUnit.NANOSECONDS.toMillis(nanos) });
             } catch (final Exception e) {
                 getLogger().error("Failed to send {} to Kafka due to {}; routing to failure", new Object[] { flowFile, e });
-                session.transfer(flowFile, REL_FAILURE);
+                session.transfer(session.penalize(flowFile), REL_FAILURE);
                 error = true;
             } finally {
                 if (error) {
@@ -534,7 +534,7 @@ public class PutKafka extends AbstractProcessor {
                 if (offset == 0L) {
                     // all of the messages failed to send. Route FlowFile to failure
                     getLogger().error("Failed to send {} to Kafka due to {}; routing to fialure", new Object[] { flowFile, pe.getCause() });
-                    session.transfer(flowFile, REL_FAILURE);
+                    session.transfer(session.penalize(flowFile), REL_FAILURE);
                 } else {
                     // Some of the messages were sent successfully. We want to split off the successful messages from the failed messages.
                     final FlowFile successfulMessages = session.clone(flowFile, 0L, offset);
@@ -545,7 +545,7 @@ public class PutKafka extends AbstractProcessor {
                         messagesSent.get(), flowFile, successfulMessages, failedMessages, pe.getCause() });
 
                     session.transfer(successfulMessages, REL_SUCCESS);
-                    session.transfer(failedMessages, REL_FAILURE);
+                    session.transfer(session.penalize(failedMessages), REL_FAILURE);
                     session.remove(flowFile);
                     session.getProvenanceReporter().send(successfulMessages, "kafka://" + topic);
                 }


[37/50] [abbrv] nifi git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/nifi

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


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

Branch: refs/heads/NIFI-730
Commit: ba72452f669b926c2af63ccd5684e0c2b793f405
Parents: a3d43d2 1d97876
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Oct 30 16:02:15 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Oct 30 16:02:15 2015 -0400

----------------------------------------------------------------------
 .../java/org/apache/nifi/action/Action.java     |  50 ++++
 .../java/org/apache/nifi/action/Component.java  |  34 +++
 .../java/org/apache/nifi/action/Operation.java  |  37 +++
 .../component/details/ComponentDetails.java     |  26 ++
 .../component/details/ExtensionDetails.java     |  26 ++
 .../details/RemoteProcessGroupDetails.java      |  26 ++
 .../nifi/action/details/ActionDetails.java      |  26 ++
 .../nifi/action/details/ConfigureDetails.java   |  30 ++
 .../nifi/action/details/ConnectDetails.java     |  40 +++
 .../apache/nifi/action/details/MoveDetails.java |  30 ++
 .../nifi/action/details/PurgeDetails.java       |  28 ++
 .../org/apache/nifi/reporting/EventAccess.java  |  13 +
 .../org/apache/nifi/util/MockEventAccess.java   |  27 ++
 .../org/apache/nifi/admin/dao/ActionDAO.java    |  12 +-
 .../java/org/apache/nifi/admin/dao/UserDAO.java |   3 +-
 .../nifi/admin/dao/impl/StandardActionDAO.java  |  93 +++---
 .../nifi/admin/dao/impl/StandardUserDAO.java    |   4 +-
 .../apache/nifi/admin/service/AuditService.java |  18 +-
 .../admin/service/action/AddActionsAction.java  |   3 +-
 .../service/action/PurgeActionsAction.java      |   3 +-
 .../service/impl/StandardAuditService.java      |  24 +-
 .../resources/nifi-administration-context.xml   |   2 +-
 .../manager/impl/ClusteredEventAccess.java      |  21 +-
 .../cluster/manager/impl/WebClusterManager.java |   2 +-
 .../apache/nifi/controller/FlowController.java  | 136 +++++----
 .../nifi/spring/FlowControllerFactoryBean.java  |   8 +
 .../src/main/resources/nifi-context.xml         |   1 +
 .../controller/StandardFlowServiceTest.java     |   5 +-
 .../nifi-framework/nifi-user-actions/pom.xml    |   7 +
 .../java/org/apache/nifi/action/Action.java     | 121 --------
 .../java/org/apache/nifi/action/Component.java  |  34 ---
 .../apache/nifi/action/FlowChangeAction.java    | 130 +++++++++
 .../java/org/apache/nifi/action/Operation.java  |  37 ---
 .../component/details/ComponentDetails.java     |  26 --
 .../component/details/ExtensionDetails.java     |  34 ---
 .../details/FlowChangeExtensionDetails.java     |  35 +++
 .../FlowChangeRemoteProcessGroupDetails.java    |  35 +++
 .../details/RemoteProcessGroupDetails.java      |  34 ---
 .../nifi/action/details/ActionDetails.java      |  26 --
 .../nifi/action/details/ConfigureDetails.java   |  52 ----
 .../nifi/action/details/ConnectDetails.java     |  90 ------
 .../details/FlowChangeConfigureDetails.java     |  55 ++++
 .../details/FlowChangeConnectDetails.java       |  97 +++++++
 .../action/details/FlowChangeMoveDetails.java   |  65 +++++
 .../action/details/FlowChangePurgeDetails.java  |  46 +++
 .../apache/nifi/action/details/MoveDetails.java |  61 ----
 .../nifi/action/details/PurgeDetails.java       |  45 ---
 .../apache/nifi/audit/ControllerAuditor.java    |  27 +-
 .../nifi/audit/ControllerServiceAuditor.java    |  43 +--
 .../org/apache/nifi/audit/FunnelAuditor.java    |   7 +-
 .../java/org/apache/nifi/audit/NiFiAuditor.java |  25 +-
 .../java/org/apache/nifi/audit/PortAuditor.java |  27 +-
 .../apache/nifi/audit/ProcessGroupAuditor.java  |  25 +-
 .../org/apache/nifi/audit/ProcessorAuditor.java |  25 +-
 .../apache/nifi/audit/RelationshipAuditor.java  |  18 +-
 .../nifi/audit/RemoteProcessGroupAuditor.java   |  35 +--
 .../apache/nifi/audit/ReportingTaskAuditor.java |  25 +-
 .../org/apache/nifi/audit/SnippetAuditor.java   |  26 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |   9 +-
 .../StandardNiFiWebConfigurationContext.java    |  13 +-
 .../apache/nifi/web/StandardNiFiWebContext.java |  13 +-
 .../nifi/web/api/ApplicationResource.java       |   5 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  20 +-
 .../nifi-pcap-processors/.gitignore             |   1 -
 .../processors/standard/AttributesToJSON.java   | 242 ++++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../standard/TestAttributesToJSON.java          | 282 +++++++++++++++++++
 67 files changed, 1781 insertions(+), 846 deletions(-)
----------------------------------------------------------------------



[10/50] [abbrv] nifi git commit: NIFI-908 Added support for SSL in JMS connections. - Added SSL context to JMS producer and consumer processors - Tony Kurc Amended patch to check SSL need by scheme and exception consistency Reviewed by Tony Kurc (tkurc

Posted by ma...@apache.org.
NIFI-908 Added support for SSL in JMS connections.
 - Added SSL context to JMS producer and consumer processors
 - Tony Kurc Amended patch to check SSL need by scheme and exception consistency
Reviewed by Tony Kurc (tkurc@apache.org)


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

Branch: refs/heads/NIFI-730
Commit: 26edab3185008f1d34647ff3c11ba8b87815de02
Parents: 8d2f9bc
Author: Luke Williamson <lu...@gmail.com>
Authored: Mon Oct 26 00:29:05 2015 -0400
Committer: Tony Kurc <tr...@gmail.com>
Committed: Mon Oct 26 00:29:05 2015 -0400

----------------------------------------------------------------------
 .../nifi/processors/standard/JmsConsumer.java   |  2 +
 .../apache/nifi/processors/standard/PutJMS.java |  2 +
 .../processors/standard/util/JmsFactory.java    | 79 ++++++++++++++++++--
 .../processors/standard/util/JmsProperties.java |  8 ++
 4 files changed, 86 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/26edab31/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
index b53d62f..461d381 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java
@@ -25,6 +25,7 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.JMS_PROPS_T
 import static org.apache.nifi.processors.standard.util.JmsProperties.JMS_PROVIDER;
 import static org.apache.nifi.processors.standard.util.JmsProperties.MESSAGE_SELECTOR;
 import static org.apache.nifi.processors.standard.util.JmsProperties.PASSWORD;
+import static org.apache.nifi.processors.standard.util.JmsProperties.SSL_CONTEXT_SERVICE;
 import static org.apache.nifi.processors.standard.util.JmsProperties.TIMEOUT;
 import static org.apache.nifi.processors.standard.util.JmsProperties.URL;
 import static org.apache.nifi.processors.standard.util.JmsProperties.USERNAME;
@@ -87,6 +88,7 @@ public abstract class JmsConsumer extends AbstractProcessor {
         descriptors.add(BATCH_SIZE);
         descriptors.add(USERNAME);
         descriptors.add(PASSWORD);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(ACKNOWLEDGEMENT_MODE);
         descriptors.add(MESSAGE_SELECTOR);
         descriptors.add(JMS_PROPS_TO_ATTRIBUTES);

http://git-wip-us.apache.org/repos/asf/nifi/blob/26edab31/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java
index dff5a6b..b8902a9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java
@@ -47,6 +47,7 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.REPLY_TO_QU
 import static org.apache.nifi.processors.standard.util.JmsProperties.TIMEOUT;
 import static org.apache.nifi.processors.standard.util.JmsProperties.URL;
 import static org.apache.nifi.processors.standard.util.JmsProperties.USERNAME;
+import static org.apache.nifi.processors.standard.util.JmsProperties.SSL_CONTEXT_SERVICE;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -122,6 +123,7 @@ public class PutJMS extends AbstractProcessor {
         descriptors.add(BATCH_SIZE);
         descriptors.add(USERNAME);
         descriptors.add(PASSWORD);
+        descriptors.add(SSL_CONTEXT_SERVICE);
         descriptors.add(MESSAGE_TYPE);
         descriptors.add(MESSAGE_PRIORITY);
         descriptors.add(REPLY_TO_QUEUE);

http://git-wip-us.apache.org/repos/asf/nifi/blob/26edab31/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsFactory.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsFactory.java
index 35a65dc..ca5df9f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsFactory.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsFactory.java
@@ -28,12 +28,15 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.DURABLE_SUB
 import static org.apache.nifi.processors.standard.util.JmsProperties.JMS_PROVIDER;
 import static org.apache.nifi.processors.standard.util.JmsProperties.MESSAGE_SELECTOR;
 import static org.apache.nifi.processors.standard.util.JmsProperties.PASSWORD;
+import static org.apache.nifi.processors.standard.util.JmsProperties.SSL_CONTEXT_SERVICE;
 import static org.apache.nifi.processors.standard.util.JmsProperties.TIMEOUT;
 import static org.apache.nifi.processors.standard.util.JmsProperties.URL;
 import static org.apache.nifi.processors.standard.util.JmsProperties.USERNAME;
 
 import java.io.IOException;
 import java.io.ObjectOutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.Map;
@@ -57,12 +60,15 @@ import javax.jms.StreamMessage;
 import javax.jms.TextMessage;
 import javax.jms.Topic;
 
-import org.apache.nifi.stream.io.ByteArrayOutputStream;
-import org.apache.nifi.processor.ProcessContext;
-
 import org.apache.activemq.ActiveMQConnectionFactory;
+import org.apache.activemq.ActiveMQSslConnectionFactory;
 import org.apache.activemq.command.ActiveMQQueue;
 import org.apache.activemq.command.ActiveMQTopic;
+import org.apache.activemq.util.URISupport;
+import org.apache.activemq.util.URISupport.CompositeData;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.stream.io.ByteArrayOutputStream;
 
 public class JmsFactory {
 
@@ -348,10 +354,43 @@ public class JmsFactory {
     }
 
     private static ConnectionFactory createConnectionFactory(final ProcessContext context) throws JMSException {
-        final String url = context.getProperty(URL).getValue();
+        final URI uri;
+        try {
+            uri = new URI(context.getProperty(URL).getValue());
+        } catch (URISyntaxException e) {
+            // Should not happen - URL was validated
+            throw new IllegalArgumentException("Validated URI [" + context.getProperty(URL) + "] was invalid", e);
+        }
         final int timeoutMillis = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
         final String provider = context.getProperty(JMS_PROVIDER).getValue();
-        return createConnectionFactory(url, timeoutMillis, provider);
+        if (uri.getScheme().equals("ssl") || (URISupport.isCompositeURI(uri) && compositeURIHasSSL(uri))) {
+            final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+            if (sslContextService == null) {
+                throw new IllegalArgumentException("Attempting to initiate SSL JMS connection and SSL Context is not set.");
+            }
+            return createSslConnectionFactory(uri, timeoutMillis, provider, sslContextService.getKeyStoreFile(),
+                    sslContextService.getKeyStorePassword(), sslContextService.getTrustStoreFile(), sslContextService.getTrustStorePassword());
+        } else {
+            return createConnectionFactory(uri, timeoutMillis, provider);
+        }
+    }
+
+    private static boolean compositeURIHasSSL(URI uri) {
+        try {
+            CompositeData compositeData = URISupport.parseComposite(uri);
+            for(URI component : compositeData.getComponents()){
+                if(component.getScheme().equals("ssl")){
+                    return true;
+                }
+            }
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Attempting to initiate JMS with invalid composite URI [" + uri + "]", e);
+        }
+        return false;
+    }
+
+    public static ConnectionFactory createConnectionFactory(final URI uri, final int timeoutMillis, final String jmsProvider) throws JMSException {
+        return createConnectionFactory(uri.toString(), timeoutMillis, jmsProvider);
     }
 
     public static ConnectionFactory createConnectionFactory(final String url, final int timeoutMillis, final String jmsProvider) throws JMSException {
@@ -366,6 +405,36 @@ public class JmsFactory {
         }
     }
 
+    public static ConnectionFactory createSslConnectionFactory(final URI uri, final int timeoutMillis, final String jmsProvider,
+            final String keystore, final String keystorePassword, final String truststore, final String truststorePassword) throws JMSException {
+        return createSslConnectionFactory(uri.toString(), timeoutMillis, jmsProvider, keystore, keystorePassword, truststore, truststorePassword);
+    }
+
+    public static ConnectionFactory createSslConnectionFactory(final String url, final int timeoutMillis, final String jmsProvider,
+                            final String keystore, final String keystorePassword, final String truststore, final String truststorePassword) throws JMSException {
+        switch (jmsProvider) {
+            case ACTIVEMQ_PROVIDER: {
+                final ActiveMQSslConnectionFactory factory = new ActiveMQSslConnectionFactory(url);
+                try {
+                    factory.setKeyStore(keystore);
+                } catch (Exception e) {
+                    throw new JMSException("Problem Setting the KeyStore: " + e.getMessage());
+                }
+                factory.setKeyStorePassword(keystorePassword);
+                try {
+                    factory.setTrustStore(truststore);
+                } catch (Exception e) {
+                    throw new JMSException("Problem Setting the TrustStore: " + e.getMessage());
+                }
+                factory.setTrustStorePassword(truststorePassword);
+                factory.setSendTimeout(timeoutMillis);
+                return factory;
+            }
+            default:
+                throw new IllegalArgumentException("Unknown JMS Provider: " + jmsProvider);
+        }
+    }
+
     public static Map<String, String> createAttributeMap(final Message message) throws JMSException {
         final Map<String, String> attributes = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/26edab31/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
index ed73569..f538624 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JmsProperties.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.standard.util;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
 
 public class JmsProperties {
 
@@ -177,4 +178,11 @@ public class JmsProperties {
             .defaultValue("1 MB")
             .build();
 
+    // JMS SSL Properties
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description("The Controller Service to use in order to obtain an SSL Context.")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
 }


[28/50] [abbrv] nifi git commit: NIFI-1051 Allowed FileSystemRepository to skip un-readable entries. The exception was caused due to basic file permissions. This fix overrides 'visitFileFailed' method of SimpleFileVisitor to log WARN message and allow Fi

Posted by ma...@apache.org.
NIFI-1051 Allowed FileSystemRepository to skip un-readable entries.
The exception was caused due to basic file permissions. This fix overrides
'visitFileFailed' method of SimpleFileVisitor to log WARN message and allow
FileSystemRepository to continue.


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

Branch: refs/heads/NIFI-730
Commit: 5c4042bd7c39dd92b1fba665c7b03c670346be8f
Parents: c4f0cb1
Author: Oleg Zhurakousky <ol...@suitcase.io>
Authored: Thu Oct 29 16:31:17 2015 -0400
Committer: Oleg Zhurakousky <ol...@suitcase.io>
Committed: Thu Oct 29 16:31:17 2015 -0400

----------------------------------------------------------------------
 .../repository/FileSystemRepository.java          |  6 ++++++
 .../repository/TestFileSystemRepository.java      | 18 ++++++++++++++++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5c4042bd/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 72a50ec..2ce0947 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -309,6 +309,12 @@ public class FileSystemRepository implements ContentRepository {
                     // the path already exists, so scan the path to find any files and update maxIndex to the max of
                     // all filenames seen.
                     Files.walkFileTree(realPath, new SimpleFileVisitor<Path>() {
+                    	
+						public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
+							LOG.warn("Content repository contains un-readable file or directory '" + file.getFileName() + "'. Skipping. ", exc);
+							return FileVisitResult.SKIP_SUBTREE;
+						}
+                    		 
                         @Override
                         public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs) throws IOException {
                             if (attrs.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/5c4042bd/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 88f572b..95e1f40 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -71,6 +71,24 @@ public class TestFileSystemRepository {
     public void shutdown() throws IOException {
         repository.shutdown();
     }
+    
+    @Test
+    public void testBogusFile() throws IOException {
+    	repository.shutdown();
+    	System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+        
+        File bogus = new File(rootFile, "bogus");
+        try {
+        	 bogus.mkdir();
+             bogus.setReadable(false);
+        
+             repository = new FileSystemRepository();
+             repository.initialize(new StandardResourceClaimManager());
+		} finally {
+			bogus.setReadable(true);
+			assertTrue(bogus.delete());
+		}
+    }
 
     @Test
     public void testCreateContentClaim() throws IOException {