diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java index 8e20b9e0e8..a7e421bd9f 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java @@ -123,6 +123,15 @@ public class ExtractGrok extends AbstractProcessor { .defaultValue("1 MB") .build(); + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() + .name("Named captures only") + .description("Only store named captures from grok") + .required(true) + .allowableValues("true", "false") + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + public static final Relationship REL_MATCH = new Relationship.Builder() .name("matched") .description("FlowFiles are routed to this relationship when the Grok Expression is successfully evaluated and the FlowFile is modified as a result") @@ -151,6 +160,7 @@ public class ExtractGrok extends AbstractProcessor { _descriptors.add(DESTINATION); _descriptors.add(CHARACTER_SET); _descriptors.add(MAX_BUFFER_SIZE); + _descriptors.add(NAMED_CAPTURES_ONLY); descriptors = Collections.unmodifiableList(_descriptors); } @@ -179,7 +189,7 @@ public class ExtractGrok extends AbstractProcessor { grok = new Grok(); grok.addPatternFromFile(context.getProperty(GROK_PATTERN_FILE).getValue()); - grok.compile(context.getProperty(GROK_EXPRESSION).getValue()); + grok.compile(context.getProperty(GROK_EXPRESSION).getValue(), context.getProperty(NAMED_CAPTURES_ONLY).asBoolean()); } @Override diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractGrok.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractGrok.java index b503c40639..b5891ad7dd 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractGrok.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractGrok.java @@ -77,7 +77,6 @@ public class TestExtractGrok { testRunner.assertNotValid(); } - @Test public void testExtractGrokWithBadGrokExpression() throws IOException { testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{TOTO"); @@ -86,4 +85,27 @@ public class TestExtractGrok { testRunner.assertNotValid(); } + @Test + public void testExtractGrokWithNamedCapturesOnly() throws IOException { + testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{COMMONAPACHELOG}"); + testRunner.setProperty(ExtractGrok.GROK_PATTERN_FILE, "src/test/resources/TestExtractGrok/patterns"); + testRunner.setProperty(ExtractGrok.NAMED_CAPTURES_ONLY, "true"); + testRunner.enqueue(GROK_LOG_INPUT); + testRunner.run(); + testRunner.assertAllFlowFilesTransferred(ExtractGrok.REL_MATCH); + final MockFlowFile matched = testRunner.getFlowFilesForRelationship(ExtractGrok.REL_MATCH).get(0); + + matched.assertAttributeEquals("grok.verb","GET"); + matched.assertAttributeEquals("grok.response","401"); + matched.assertAttributeEquals("grok.bytes","12846"); + matched.assertAttributeEquals("grok.clientip","64.242.88.10"); + matched.assertAttributeEquals("grok.auth","-"); + matched.assertAttributeEquals("grok.timestamp","07/Mar/2004:16:05:49 -0800"); + matched.assertAttributeEquals("grok.request","/twiki/bin/edit/Main/Double_bounce_sender?topicparent=Main.ConfigurationVariables"); + matched.assertAttributeEquals("grok.httpversion","1.1"); + + matched.assertAttributeNotExists("grok.INT"); + matched.assertAttributeNotExists("grok.BASE10NUM"); + matched.assertAttributeNotExists("grok.COMMONAPACHELOG"); + } }