Repository: nifi
Updated Branches:
  refs/heads/master 50667ed22 -> 066accc27


NIFI-3086 Updated TailFile.java to deal with windows paths that were breaking 
the regex parsing Fixed documentation misspelling in TailFile.java Fixed 
assertions in several tests in TestTailFile that were dependent on the received 
order of flowfiles, this was differing between Windows and MacOS/Linux

This closes #1264

Signed-off-by: jpercivall <[email protected]>


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

Branch: refs/heads/master
Commit: 066accc274540bb2bfd392cffd1e86bb9ff48bb0
Parents: 50667ed
Author: Jeff Storck <[email protected]>
Authored: Tue Nov 22 22:01:22 2016 -0500
Committer: jpercivall <[email protected]>
Committed: Wed Nov 23 00:01:12 2016 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/TailFile.java      | 12 ++-
 .../nifi/processors/standard/TestTailFile.java  | 82 +++++++++++++-------
 2 files changed, 63 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/066accc2/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
----------------------------------------------------------------------
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
index 3b71609..3553ce8 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java
@@ -136,7 +136,7 @@ public class TailFile extends AbstractProcessor {
     static final PropertyDescriptor MODE = new PropertyDescriptor.Builder()
             .name("tail-mode")
             .displayName("Tailing mode")
-            .description("Mode to use: single file will tail only one gile, 
multiple file will look for a list of file. In Multiple mode"
+            .description("Mode to use: single file will tail only one file, 
multiple file will look for a list of file. In Multiple mode"
                     + " the Base directory is required.")
             .expressionLanguageSupported(false)
             .required(true)
@@ -415,7 +415,15 @@ public class TailFile extends AbstractProcessor {
         Collection<File> files = FileUtils.listFiles(new File(baseDir), null, 
isRecursive);
         List<String> result = new ArrayList<String>();
 
-        String fullRegex = baseDir.endsWith(File.separator) ? baseDir + 
fileRegex : baseDir + File.separator + fileRegex;
+        String baseDirNoTrailingSeparator = baseDir.endsWith(File.separator) ? 
baseDir.substring(0, baseDir.length() -1) : baseDir;
+        final String fullRegex;
+        if (File.separator.equals("/")) {
+            // handle unix-style paths
+            fullRegex = baseDirNoTrailingSeparator + File.separator + 
fileRegex;
+        } else {
+            // handle windows-style paths, need to quote backslash characters
+            fullRegex = baseDirNoTrailingSeparator + 
Pattern.quote(File.separator) + fileRegex;
+        }
         Pattern p = Pattern.compile(fullRegex);
 
         for(File file : files) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/066accc2/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
----------------------------------------------------------------------
diff --git 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
index 5e49a75..0cda3f0 100644
--- 
a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
+++ 
b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java
@@ -24,6 +24,8 @@ import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.util.Optional;
+import java.util.regex.Pattern;
 
 import org.apache.nifi.processors.standard.TailFile.TailFileState;
 import org.apache.nifi.util.MockFlowFile;
@@ -547,7 +549,13 @@ public class TestTailFile {
     public void testMultipleFiles() throws IOException, InterruptedException {
         runner.setProperty(TailFile.BASE_DIRECTORY, "target");
         runner.setProperty(TailFile.MODE, TailFile.MODE_MULTIFILE);
-        runner.setProperty(TailFile.FILENAME, "(testDir/)?log(ging)?.txt");
+        final String fileRegex;
+        if (File.separator.equals("/")) {
+            fileRegex = "(testDir/)?log(ging)?.txt";
+        } else {
+            fileRegex = "(testDir" + Pattern.quote(File.separator) + 
")?log(ging)?.txt";
+        }
+        runner.setProperty(TailFile.FILENAME, fileRegex);
         runner.setProperty(TailFile.ROLLING_FILENAME_PATTERN, "${filename}.?");
         runner.setProperty(TailFile.START_POSITION, 
TailFile.START_CURRENT_FILE);
         runner.setProperty(TailFile.RECURSIVE, "true");
@@ -570,12 +578,18 @@ public class TestTailFile {
 
         runner.run(1);
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 3);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hey\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertAttributeEquals("tailfile.original.path",
 thirdFile.getPath());
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("hi\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertAttributeEquals("tailfile.original.path",
 otherFile.getPath());
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(2).assertContentEquals("hello\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(2).assertAttributeEquals("tailfile.original.path",
 file.getPath());
+        Optional<MockFlowFile> thirdFileFF = 
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS)
+                .stream().filter(mockFlowFile -> 
mockFlowFile.isAttributeEqual("tailfile.original.path", 
thirdFile.getPath())).findFirst();
+        assertTrue(thirdFileFF.isPresent());
+        thirdFileFF.get().assertContentEquals("hey\n");
+        Optional<MockFlowFile> otherFileFF = 
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS)
+                .stream().filter(mockFlowFile -> 
mockFlowFile.isAttributeEqual("tailfile.original.path", 
otherFile.getPath())).findFirst();
+        assertTrue(otherFileFF.isPresent());
+        otherFileFF.get().assertContentEquals("hi\n");
+        Optional<MockFlowFile> fileFF = 
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS)
+                .stream().filter(mockFlowFile -> 
mockFlowFile.isAttributeEqual("tailfile.original.path", 
file.getPath())).findFirst();
+        assertTrue(fileFF.isPresent());
+        fileFF.get().assertContentEquals("hello\n");
         runner.clearTransferState();
 
         otherRaf.write("world!".getBytes());
@@ -613,11 +627,11 @@ public class TestTailFile {
         runner.run(1);
 
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 5);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("3\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("world!");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(2).assertContentEquals("2\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(3).assertContentEquals("world");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(4).assertContentEquals("1\n");
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("3\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("world!")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("2\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("world")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("1\n")));
     }
 
     @Test
@@ -625,7 +639,13 @@ public class TestTailFile {
         runner.setVariable("vrBaseDirectory", "target");
         runner.setProperty(TailFile.BASE_DIRECTORY, "${vrBaseDirectory}");
         runner.setProperty(TailFile.MODE, TailFile.MODE_MULTIFILE);
-        runner.setVariable("vrFilename", "(testDir/)?log(ging)?.txt");
+        final String fileRegex;
+        if (File.separator.equals("/")) {
+            fileRegex = "(testDir/)?log(ging)?.txt";
+        } else {
+            fileRegex = "(testDir" + Pattern.quote(File.separator) + 
")?log(ging)?.txt";
+        }
+        runner.setVariable("vrFilename", fileRegex);
         runner.setProperty(TailFile.FILENAME, "${vrFilename}");
         runner.setProperty(TailFile.ROLLING_FILENAME_PATTERN, "${filename}.?");
         runner.setProperty(TailFile.START_POSITION, 
TailFile.START_CURRENT_FILE);
@@ -669,10 +689,14 @@ public class TestTailFile {
 
         runner.run(1);
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 2);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hey\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertAttributeEquals("tailfile.original.path",
 myOtherFile.getPath());
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("hello\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertAttributeEquals("tailfile.original.path",
 file.getPath());
+        Optional<MockFlowFile> myOtherFileFF = 
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS)
+                .stream().filter(mockFlowFile -> 
mockFlowFile.isAttributeEqual("tailfile.original.path", 
myOtherFile.getPath())).findFirst();
+        assertTrue(myOtherFileFF.isPresent());
+        myOtherFileFF.get().assertContentEquals("hey\n");
+        Optional<MockFlowFile> fileFF = 
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS)
+                .stream().filter(mockFlowFile -> 
mockFlowFile.isAttributeEqual("tailfile.original.path", 
file.getPath())).findFirst();
+        assertTrue(fileFF.isPresent());
+        fileFF.get().assertContentEquals("hello\n");
         runner.clearTransferState();
 
         myOtherRaf.write("guys".getBytes());
@@ -700,10 +724,10 @@ public class TestTailFile {
         runner.run(1);
 
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 4);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("guys");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("2\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(2).assertContentEquals("world");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(3).assertContentEquals("1\n");
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("guys")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("2\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("world")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("1\n")));
     }
 
     @Test
@@ -736,8 +760,8 @@ public class TestTailFile {
 
         runner.run(1, false);
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 2);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("hey\n");
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hello\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hey\n")));
         runner.clearTransferState();
 
         multiChangeFirstRaf.write("hey2\n".getBytes());
@@ -747,8 +771,8 @@ public class TestTailFile {
         runner.run(1, false);
 
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 2);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello2\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("hey2\n");
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hello2\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hey2\n")));
         runner.clearTransferState();
 
         multiChangeFirstRaf.write("hey3\n".getBytes());
@@ -781,10 +805,10 @@ public class TestTailFile {
         multiChangeSndRaf.close();
 
         runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 4);
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello3\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(1).assertContentEquals("hello\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(2).assertContentEquals("hey3\n");
-        
runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(3).assertContentEquals("hey\n");
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hello3\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hello\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hey3\n")));
+        
assertTrue(runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).stream().anyMatch(mockFlowFile
 -> mockFlowFile.isContentEqual("hey\n")));
         runner.clearTransferState();
     }
 

Reply via email to