You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by pv...@apache.org on 2017/06/22 21:31:44 UTC

nifi git commit: NIFI-4095 Changed minimum capture group count in ExtractText from 1 to 0. Added unit test and removed obsolete test. Added custom validation to enforce capture group if "include capture group 0" is false.

Repository: nifi
Updated Branches:
  refs/heads/master 5c755c006 -> 253ea2e73


NIFI-4095 Changed minimum capture group count in ExtractText from 1 to 0.
Added unit test and removed obsolete test.
Added custom validation to enforce capture group if "include capture group 0" is false.


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

Branch: refs/heads/master
Commit: 253ea2e73bd271e82dcfd6c706f679ddad014101
Parents: 5c755c0
Author: Andy LoPresto <al...@apache.org>
Authored: Tue Jun 20 16:23:19 2017 -0400
Committer: Pierre Villard <pi...@gmail.com>
Committed: Thu Jun 22 23:31:35 2017 +0200

----------------------------------------------------------------------
 .../nifi/processors/standard/ExtractText.java   | 73 ++++++++++++++------
 .../processors/standard/TestExtractText.java    | 44 +++++++++---
 2 files changed, 86 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/253ea2e7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
index c4e55e7..692446b 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -31,7 +32,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.nifi.annotation.behavior.DynamicProperty;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -43,6 +43,9 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -62,26 +65,26 @@ import org.apache.nifi.stream.io.StreamUtils;
 @Tags({"evaluate", "extract", "Text", "Regular Expression", "regex"})
 @CapabilityDescription(
         "Evaluates one or more Regular Expressions against the content of a FlowFile.  "
-        + "The results of those Regular Expressions are assigned to FlowFile Attributes.  "
-        + "Regular Expressions are entered by adding user-defined properties; "
-        + "the name of the property maps to the Attribute Name into which the result will be placed.  "
-        + "The first capture group, if any found, will be placed into that attribute name."
-        + "But all capture groups, including the matching string sequence itself will also be "
-        + "provided at that attribute name with an index value provided, with the exception of a capturing group "
-        + "that is optional and does not match - for example, given the attribute name \"regex\" and expression "
-        + "\"abc(def)?(g)\" we would add an attribute \"regex.1\" with a value of \"def\" if the \"def\" matched. If "
-        + "the \"def\" did not match, no attribute named \"regex.1\" would be added but an attribute named \"regex.2\" "
-        + "with a value of \"g\" will be added regardless."
-        + "The value of the property must be a valid Regular Expressions with one or more capturing groups. "
-        + "If the Regular Expression matches more than once, only the first match will be used unless the property "
-        + "enabling repeating capture group is set to true. "
-        + "If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. "
-        + "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' "
-        + "and no attributes will be applied to the FlowFile.")
+                + "The results of those Regular Expressions are assigned to FlowFile Attributes.  "
+                + "Regular Expressions are entered by adding user-defined properties; "
+                + "the name of the property maps to the Attribute Name into which the result will be placed.  "
+                + "The first capture group, if any found, will be placed into that attribute name."
+                + "But all capture groups, including the matching string sequence itself will also be "
+                + "provided at that attribute name with an index value provided, with the exception of a capturing group "
+                + "that is optional and does not match - for example, given the attribute name \"regex\" and expression "
+                + "\"abc(def)?(g)\" we would add an attribute \"regex.1\" with a value of \"def\" if the \"def\" matched. If "
+                + "the \"def\" did not match, no attribute named \"regex.1\" would be added but an attribute named \"regex.2\" "
+                + "with a value of \"g\" will be added regardless."
+                + "The value of the property must be a valid Regular Expressions with one or more capturing groups. "
+                + "If the Regular Expression matches more than once, only the first match will be used unless the property "
+                + "enabling repeating capture group is set to true. "
+                + "If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. "
+                + "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' "
+                + "and no attributes will be applied to the FlowFile.")
 @DynamicProperty(name = "A FlowFile attribute", value = "A Regular Expression with one or more capturing group",
         description = "The first capture group, if any found, will be placed into that attribute name."
-        + "But all capture groups, including the matching string sequence itself will also be "
-        + "provided at that attribute name with an index value provided.")
+                + "But all capture groups, including the matching string sequence itself will also be "
+                + "provided at that attribute name with an index value provided.")
 public class ExtractText extends AbstractProcessor {
 
     public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
@@ -259,12 +262,38 @@ public class ExtractText extends AbstractProcessor {
         return new PropertyDescriptor.Builder()
                 .name(propertyDescriptorName)
                 .expressionLanguageSupported(false)
-                .addValidator(StandardValidators.createRegexValidator(1, 40, true))
+                .addValidator(StandardValidators.createRegexValidator(0, 40, true))
                 .required(false)
                 .dynamic(true)
                 .build();
     }
 
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> problems = new ArrayList<>(super.customValidate(validationContext));
+
+        // If the capture group zero is not going to be included, each dynamic property must have at least one group
+        final boolean includeCaptureGroupZero = validationContext.getProperty(INCLUDE_CAPTURE_GROUP_ZERO).getValue().equalsIgnoreCase("true");
+        getLogger().debug("Include capture group zero is " + includeCaptureGroupZero);
+        if (!includeCaptureGroupZero) {
+            final Validator oneGroupMinimumValidator = StandardValidators.createRegexValidator(1, 40, true);
+            for (Map.Entry<PropertyDescriptor, String> prop : validationContext.getProperties().entrySet()) {
+                PropertyDescriptor pd = prop.getKey();
+                if (pd.isDynamic()) {
+                    String value = validationContext.getProperty(pd).getValue();
+                    getLogger().debug("Evaluating dynamic property " + pd.getDisplayName() + " (" + pd.getName() + ") with value " + value);
+                    ValidationResult result = oneGroupMinimumValidator.validate(pd.getDisplayName(), value, validationContext);
+                    getLogger().debug("Validation result: " + result.toString());
+                    if (!result.isValid()) {
+                        problems.add(result);
+                    }
+                }
+            }
+        }
+
+        return problems;
+    }
+
     @OnScheduled
     public final void onScheduled(final ProcessContext context) throws IOException {
         final Map<String, Pattern> compiledPatternsMap = new HashMap<>();
@@ -338,7 +367,7 @@ public class ExtractText extends AbstractProcessor {
                 final String baseKey = entry.getKey();
                 int start = j == 0 ? startGroupIdx : 1;
                 for (int i = start; i <= matcher.groupCount(); i++) {
-                    final String key = new StringBuilder(baseKey).append(".").append(i+j).toString();
+                    final String key = new StringBuilder(baseKey).append(".").append(i + j).toString();
                     String value = matcher.group(i);
                     if (value != null && !value.isEmpty()) {
                         if (value.length() > maxCaptureGroupLength) {
@@ -351,7 +380,7 @@ public class ExtractText extends AbstractProcessor {
                     }
                 }
                 j += matcher.groupCount();
-                if(!context.getProperty(ENABLE_REPEATING_CAPTURE_GROUP).asBoolean()) {
+                if (!context.getProperty(ENABLE_REPEATING_CAPTURE_GROUP).asBoolean()) {
                     break;
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/253ea2e7/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java
index 93d5147..e323c2d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.UnsupportedEncodingException;
 import java.util.Set;
 import java.util.regex.Pattern;
-
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
@@ -216,14 +215,6 @@ public class TestExtractText {
         out.assertAttributeEquals("regex.result7", null);
     }
 
-    @Test(expected = java.lang.AssertionError.class)
-    public void testNoCaptureGroups() throws UnsupportedEncodingException {
-        final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
-        testRunner.setProperty("regex.result1", ".*");
-        testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
-        testRunner.run();
-    }
-
     @Test
     public void testNoFlowFile() throws UnsupportedEncodingException {
         final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
@@ -422,4 +413,39 @@ public class TestExtractText {
         out.assertAttributeNotExists(attributeKey + ".0");
         out.assertAttributeEquals(attributeKey, SAMPLE_STRING);
     }
+
+    @Test
+    public void testShouldAllowNoCaptureGroups() throws Exception {
+        // Arrange
+        final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
+        final String attributeKey = "regex.result";
+        testRunner.setProperty(attributeKey, "(?s).*");
+
+        // Act
+        testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
+        testRunner.run();
+
+        // Assert
+        testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
+        final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
+
+        // There is no global capture group, so only "key.0" exists
+        out.assertAttributeNotExists(attributeKey);
+        out.assertAttributeEquals(attributeKey + ".0", SAMPLE_STRING);
+    }
+
+    @Test(expected = java.lang.AssertionError.class)
+    public void testShouldNotAllowNoCaptureGroupsIfZeroDisabled() throws Exception {
+        // Arrange
+        final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
+        testRunner.setProperty(ExtractText.INCLUDE_CAPTURE_GROUP_ZERO, "false");
+        final String attributeKey = "regex.result";
+        testRunner.setProperty(attributeKey, "(?s).*");
+
+        // Act
+        testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
+
+        // Validation should fail because nothing will match
+        testRunner.run();
+    }
 }