[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164636057 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() --- End diff -- I now recall my reasoning on this. The common way to use grok is to have most of the regexes in one or more grok pattern files, where they are easier to test then in an attribute property box. The expressions are then referenced by name in the expression passed in for evaluation, as in %{EX_1}. or even {EX_1} %{EX_2}. If someone really does want to do an interesting regex in line, the delimiter can be multiple characters, so it is easy to make a unique one like "<>". If I wanted to locally define a set of regexes, I would could also also use the ExtractText processor. On the other hand, if we go with putting each expression in a separate attributes, the attributes need to be named in a way that can be sorted since we commit to order of evaluation. If the user wants to insert a new expression into the middle of the list, they have to (on average) recreate half of the rest of the attributes with new names. In my own use case of building a classifier, i am often tweaking the list, so having to recreate my attributes would be a nuisance. PS: I will make the delimiter optional, with no default. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164564814 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -209,78 +287,94 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try { final byte[] byteBuffer = buffer; -session.read(flowFile, new InputStreamCallback() { -@Override -public void process(InputStream in) throws IOException { -StreamUtils.fillBuffer(in, byteBuffer, false); -} -}); +session.read(flowFile, in -> StreamUtils.fillBuffer(in, byteBuffer, false)); final long len = Math.min(byteBuffer.length, flowFile.getSize()); contentString = new String(byteBuffer, 0, (int) len, charset); } finally { bufferQueue.offer(buffer); } -final Match gm = grok.match(contentString); -gm.captures(); - -if (gm.toMap().isEmpty()) { -session.transfer(flowFile, REL_NO_MATCH); -getLogger().info("Did not match any Grok Expressions for FlowFile {}", new Object[]{flowFile}); -return; -} - -final ObjectMapper objectMapper = new ObjectMapper(); -switch (context.getProperty(DESTINATION).getValue()) { -case FLOWFILE_ATTRIBUTE: -MapgrokResults = new HashMap<>(); -for (Map.Entry entry : gm.toMap().entrySet()) { -if (null != entry.getValue()) { -grokResults.put("grok." + entry.getKey(), entry.getValue().toString()); +try{ +for (Grok grok : grokList) { +final Match gm = grok.match(contentString); +gm.captures(); +final Map localResults = gm.toMap(); +if (!localResults.isEmpty()) { + matchedExpressionList.add(grok.getOriginalGrokPattern()); +results.putAll(localResults); +if (breakOnFirstMatch) { +break; } } +} -flowFile = session.putAllAttributes(flowFile, grokResults); -session.getProvenanceReporter().modifyAttributes(flowFile); -session.transfer(flowFile, REL_MATCH); -getLogger().info("Matched {} Grok Expressions and added attributes to FlowFile {}", new Object[]{grokResults.size(), flowFile}); - -break; -case FLOWFILE_CONTENT: -FlowFile conFlowfile = session.write(flowFile, new StreamCallback() { -@Override -public void process(InputStream in, OutputStream out) throws IOException { - out.write(objectMapper.writeValueAsBytes(gm.toMap())); -} -}); -conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), APPLICATION_JSON); -session.getProvenanceReporter().modifyContent(conFlowfile, "Replaced content with parsed Grok fields and values", stopWatch.getElapsed(TimeUnit.MILLISECONDS)); -session.transfer(conFlowfile, REL_MATCH); +if (results.isEmpty()) { +session.transfer(flowFile, REL_NO_MATCH); +getLogger().debug("Did not match any Grok Expression for FlowFile {}", new Object[]{flowFile}); +return; +} + +String matchedExpressions = StringUtils.join(matchedExpressionList, expressionSeparator); +flowFile = session.putAttribute(flowFile, matchedExpressionAttribute, matchedExpressions); -break; +switch (context.getProperty(DESTINATION).getValue()) { +case FLOWFILE_ATTRIBUTE: +Map grokResults = new HashMap<>(); +for (Map.Entry entry : results.entrySet()) { +if (null != entry.getValue()) { +grokResults.put(resultPrefix + entry.getKey(), entry.getValue().toString()); +} +} +flowFile = session.putAllAttributes(flowFile, grokResults); + session.getProvenanceReporter().modifyAttributes(flowFile); +session.transfer(flowFile,
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164564264 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -181,15 +243,28 @@ public void onStopped() { @OnScheduled public void onScheduled(final ProcessContext context) throws GrokException { +grokList.clear(); for (int i = 0; i < context.getMaxConcurrentTasks(); i++) { final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); final byte[] buffer = new byte[maxBufferSize]; bufferQueue.add(buffer); } -grok = new Grok(); - grok.addPatternFromFile(context.getProperty(GROK_PATTERN_FILE).getValue()); -grok.compile(context.getProperty(GROK_EXPRESSION).getValue(), context.getProperty(NAMED_CAPTURES_ONLY).asBoolean()); +resultPrefix = context.getProperty(RESULT_PREFIX).getValue(); +breakOnFirstMatch = context.getProperty(BREAK_ON_FIRST_MATCH).asBoolean() ; +matchedExpressionAttribute = context.getProperty(MATCHED_EXP_ATTR).getValue(); +expressionSeparator = context.getProperty(EXPRESSION_SEPARATOR).getValue(); + +String patterns = context.getProperty(GROK_EXPRESSION).getValue(); +for (String patternName : patterns.split(expressionSeparator)) { +Grok grok = new Grok(); +final String patternFileListString = context.getProperty(GROK_PATTERN_FILE).getValue(); +for (String patternFile : patternFileListString.split(PATTERN_FILE_LIST_SEPARATOR)) { +grok.addPatternFromFile(patternFile); --- End diff -- hmmm... ok. My feeling would be if they want to put spaces in their list they should include them in separator, but your way is more tolerant. Will fix. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164562467 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -156,14 +209,23 @@ final List _descriptors = new ArrayList<>(); _descriptors.add(GROK_EXPRESSION); +_descriptors.add(EXPRESSION_SEPARATOR); _descriptors.add(GROK_PATTERN_FILE); _descriptors.add(DESTINATION); _descriptors.add(CHARACTER_SET); _descriptors.add(MAX_BUFFER_SIZE); _descriptors.add(NAMED_CAPTURES_ONLY); +_descriptors.add(RESULT_PREFIX); +_descriptors.add(BREAK_ON_FIRST_MATCH); +_descriptors.add(MATCHED_EXP_ATTR); descriptors = Collections.unmodifiableList(_descriptors); } +private String resultPrefix = ""; --- End diff -- I agree I should conform to project style. will fix. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164561259 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -142,10 +194,11 @@ .description("FlowFiles are routed to this relationship when no provided Grok Expression matches the content of the FlowFile") .build(); + private final static List descriptors; private final static Set relationships; -private volatile Grok grok = new Grok(); +private volatile List grokList = new ArrayList<>(); --- End diff -- ok, i will create a new list. In practice, it is never modified while running multi-threaded. but it is an easy and computationally cheap fix for safer code. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164560737 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() +.name(EXP_SEPARATOR_KEY) +.description("character to use to separate multiple grok expressions ") +.required(true) +.defaultValue(",") +.addValidator(StandardValidators.NON_BLANK_VALIDATOR) +.build(); + + +public static final PropertyDescriptor MATCHED_EXP_ATTR = new PropertyDescriptor.Builder() +.name(MATCHED_EXP_ATTR_KEY) +.description("Name of attribute to receive the name(s) of the matched expression(s).") +.required(true) +.defaultValue("matched_expression") +.addValidator(Validator.VALID) --- End diff -- oops. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164560550 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() --- End diff -- good point about the backward compatibility. I had imagined that users could select a separator if needed in that case.I now realize that is not reasonable. I did not originally go with user-defined properties cause I was not sure how they would be ordered. Are they delivered in the same order declared, or do I need to tell the user to name them in sortable order? ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164559686 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + --- End diff -- yep br ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164559463 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") --- End diff -- glad to get early warning that it was not intelligible. I will add a more complete just FYI: in my use case. i am using this processor as a classifier. I arrange my expressions, so that the first match it makes will be the best match.. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164559047 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") --- End diff -- my bad. i will do something more expressive. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164558925 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -70,33 +65,51 @@ @Tags({"grok", "log", "text", "parse", "delimit", "extract"}) @CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " + "adding the results as attributes or replacing the content of the FlowFile with a JSON " + -"notation of the matched content") +"notation of the matched content\n" + +"uses https://github.com/thekrakken/java-grok.;) @WritesAttributes({ -@WritesAttribute(attribute = "grok.XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + -"will be added as an attribute, prefixed with \"grok.\" For example," + -"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"grok.timestamp\"")}) +@WritesAttribute(attribute = "{result prefix}XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + +"will be added as an attribute, prefixed with \"{result prefix}\" For example," + +"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"{result prefix}timestamp\""), + +@WritesAttribute(attribute = "ExtractGrok.exception", description = "if an error occurs, an exception will be written to this attribute, " + +"and the flow routed to 'unmatched' ") +}) public class ExtractGrok extends AbstractProcessor { public static final String FLOWFILE_ATTRIBUTE = "flowfile-attribute"; public static final String FLOWFILE_CONTENT = "flowfile-content"; -private static final String APPLICATION_JSON = "application/json"; - +public static final String APPLICATION_JSON = "application/json"; +public static final String GROK_EXPRESSION_KEY = "Grok Expression"; +public static final String GROK_PATTERN_FILE_KEY = "Grok Pattern file"; +public static final String DESTINATION_KEY = "Destination"; +public static final String CHARACTER_SET_KEY = "Character Set"; +public static final String MAXIMUM_BUFFER_SIZE_KEY = "Maximum Buffer Size"; +public static final String NAMED_CAPTURES_ONLY_KEY = "Named captures only"; +public static final String SINGLE_MATCH_KEY = "Single Match"; +public static final String RESULT_PREFIX_KEY = "result prefix"; +public static final String MATCHED_EXP_ATTR_KEY = "matched expression attribute"; +public static final String EXP_SEPARATOR_KEY = "expression-separator"; +public static final String PATTERN_FILE_LIST_SEPARATOR = ","; + +//properties public static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor.Builder() -.name("Grok Expression") -.description("Grok expression") +.name(GROK_EXPRESSION_KEY) --- End diff -- just because sometimes (maybe not in this processor) the literals for the properties do get re-used. I like consistency, even if in a particular situation it might not add much. If the NIFI style is to not to do it this way, I will happily use literals. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user charlesporter commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164558402 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -70,33 +65,51 @@ @Tags({"grok", "log", "text", "parse", "delimit", "extract"}) @CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " + "adding the results as attributes or replacing the content of the FlowFile with a JSON " + -"notation of the matched content") +"notation of the matched content\n" + --- End diff -- thanks, i will use instead of \n. and add the notice. (BTW: It is same implementation as was being used before, without credit) ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164502487 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") --- End diff -- I'm not sure that I understand what 'grokList' means here. I think it's intended to refer to the name of a variable in the Processor? We need to keep in mind that the user of this Processor will not be looking at the code so we need to explain this in a way that makes sense to the user. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164509747 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -70,33 +65,51 @@ @Tags({"grok", "log", "text", "parse", "delimit", "extract"}) @CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " + "adding the results as attributes or replacing the content of the FlowFile with a JSON " + -"notation of the matched content") +"notation of the matched content\n" + --- End diff -- I would avoid using the \n here as this is rendered in a couple of places in HTML and we don't guarantee that it will render as expected. If we are going to explicitly indicate which library is being used, we should also be very clear that this is the implementation that is currently used and that it could change at any time without notice. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164510291 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -70,33 +65,51 @@ @Tags({"grok", "log", "text", "parse", "delimit", "extract"}) @CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " + "adding the results as attributes or replacing the content of the FlowFile with a JSON " + -"notation of the matched content") +"notation of the matched content\n" + +"uses https://github.com/thekrakken/java-grok.;) @WritesAttributes({ -@WritesAttribute(attribute = "grok.XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + -"will be added as an attribute, prefixed with \"grok.\" For example," + -"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"grok.timestamp\"")}) +@WritesAttribute(attribute = "{result prefix}XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + +"will be added as an attribute, prefixed with \"{result prefix}\" For example," + +"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"{result prefix}timestamp\""), + +@WritesAttribute(attribute = "ExtractGrok.exception", description = "if an error occurs, an exception will be written to this attribute, " + +"and the flow routed to 'unmatched' ") --- End diff -- This should read "and the FlowFile routed" rather than "and the flow routed". In such a case, though, we should not be routing to 'unmatched' but rather we should route to a failure relationship. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164502184 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + --- End diff -- We should probably avoid using \n and \t in the description here and instead just write in more of a paragraph form. Since this is rendered in HTML and in a couple of different places, there is no guarantee that this will be rendered the way that it's intended. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164503277 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -156,14 +209,23 @@ final List _descriptors = new ArrayList<>(); _descriptors.add(GROK_EXPRESSION); +_descriptors.add(EXPRESSION_SEPARATOR); _descriptors.add(GROK_PATTERN_FILE); _descriptors.add(DESTINATION); _descriptors.add(CHARACTER_SET); _descriptors.add(MAX_BUFFER_SIZE); _descriptors.add(NAMED_CAPTURES_ONLY); +_descriptors.add(RESULT_PREFIX); +_descriptors.add(BREAK_ON_FIRST_MATCH); +_descriptors.add(MATCHED_EXP_ATTR); descriptors = Collections.unmodifiableList(_descriptors); } +private String resultPrefix = ""; --- End diff -- This is not thread-safe. Since each invocation of the Processor may use a different thread, we must ensure that any member variable is stored in a thread-safe manner. Given that these variables appear to be set only in the @OnScheduled method and are not mutable objects, it should be sufficient to mark them as volatile. It is probably best, though, to move them up in the codebase with the other member variables, as that's the de-facto standard with nifi processors. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164503563 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -181,15 +243,28 @@ public void onStopped() { @OnScheduled public void onScheduled(final ProcessContext context) throws GrokException { +grokList.clear(); for (int i = 0; i < context.getMaxConcurrentTasks(); i++) { final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); final byte[] buffer = new byte[maxBufferSize]; bufferQueue.add(buffer); } -grok = new Grok(); - grok.addPatternFromFile(context.getProperty(GROK_PATTERN_FILE).getValue()); -grok.compile(context.getProperty(GROK_EXPRESSION).getValue(), context.getProperty(NAMED_CAPTURES_ONLY).asBoolean()); +resultPrefix = context.getProperty(RESULT_PREFIX).getValue(); +breakOnFirstMatch = context.getProperty(BREAK_ON_FIRST_MATCH).asBoolean() ; +matchedExpressionAttribute = context.getProperty(MATCHED_EXP_ATTR).getValue(); +expressionSeparator = context.getProperty(EXPRESSION_SEPARATOR).getValue(); + +String patterns = context.getProperty(GROK_EXPRESSION).getValue(); +for (String patternName : patterns.split(expressionSeparator)) { +Grok grok = new Grok(); +final String patternFileListString = context.getProperty(GROK_PATTERN_FILE).getValue(); +for (String patternFile : patternFileListString.split(PATTERN_FILE_LIST_SEPARATOR)) { +grok.addPatternFromFile(patternFile); --- End diff -- It probably makes sense to call trim() on this value before passing it into the Grok object so that if the user enters something like "abc, xyz, 123" we don't pass in the white space. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164511690 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() --- End diff -- I am wondering if it makes more sense to avoid using a delimiter at all here and instead support user-defined properties to indicate additional expressions. So we would keep the existing property, which is required. Then if the user adds a new property, say "my additional grok expression" the value of that would be a grok expression to match as well. Thoughts? ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164503950 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -142,10 +194,11 @@ .description("FlowFiles are routed to this relationship when no provided Grok Expression matches the content of the FlowFile") .build(); + private final static List descriptors; private final static Set relationships; -private volatile Grok grok = new Grok(); +private volatile List grokList = new ArrayList<>(); --- End diff -- This ArrayList is modified once created, so marking it as volatile is not sufficient from a thread-safety point of view. We would need to either create a new list each time instead of calling `clear()` in the @OnScheduled method or make it a synchronized list. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164500833 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() +.name(EXP_SEPARATOR_KEY) +.description("character to use to separate multiple grok expressions ") +.required(true) +.defaultValue(",") +.addValidator(StandardValidators.NON_BLANK_VALIDATOR) +.build(); + + +public static final PropertyDescriptor MATCHED_EXP_ATTR = new PropertyDescriptor.Builder() +.name(MATCHED_EXP_ATTR_KEY) +.description("Name of attribute to receive the name(s) of the matched expression(s).") +.required(true) +.defaultValue("matched_expression") +.addValidator(Validator.VALID) --- End diff -- This should not support empty string. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164501620 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") --- End diff -- Can you explain better what this property is meant to do? How does the behavior of the processor change if i select 'true' vs. if I select 'false'? ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164501376 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -107,31 +120,70 @@ .build(); public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() -.name("Character Set") +.name(CHARACTER_SET_KEY) .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") +.name(MAXIMUM_BUFFER_SIZE_KEY) .description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .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") + public static final PropertyDescriptor NAMED_CAPTURES_ONLY = new PropertyDescriptor.Builder() +.name(NAMED_CAPTURES_ONLY_KEY) +.description("Only store named captures from grokList") .required(true) .allowableValues("true", "false") .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .defaultValue("false") .build(); +public static final PropertyDescriptor BREAK_ON_FIRST_MATCH = new PropertyDescriptor.Builder() +.name(SINGLE_MATCH_KEY) +.description("Stop on first matched expression.") +.required(true) +.allowableValues("true", "false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.defaultValue("true") +.build(); + +public static final PropertyDescriptor RESULT_PREFIX = new PropertyDescriptor.Builder() +.name(RESULT_PREFIX_KEY) +.description("Value to prefix attribute results with (avoid collisions with existing properties)" + +"\n\t (Does not apply when results returned as content)" + +"\n\t (May be empty, the dot (.) separator is not implied)") +.required(true) +.defaultValue("grok.") +.addValidator(Validator.VALID) +.build(); + + +public static final PropertyDescriptor EXPRESSION_SEPARATOR = new PropertyDescriptor.Builder() --- End diff -- This would break backward compatibility of this processor. It will be very common to have a comma in the Grok Expression, and this change would result in different behavior for the processor. If we use this approach, then we would have to make it an optional property (i.e., `required(false)`) and we would have to not set a default value. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164499982 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -70,33 +65,51 @@ @Tags({"grok", "log", "text", "parse", "delimit", "extract"}) @CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " + "adding the results as attributes or replacing the content of the FlowFile with a JSON " + -"notation of the matched content") +"notation of the matched content\n" + +"uses https://github.com/thekrakken/java-grok.;) @WritesAttributes({ -@WritesAttribute(attribute = "grok.XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + -"will be added as an attribute, prefixed with \"grok.\" For example," + -"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"grok.timestamp\"")}) +@WritesAttribute(attribute = "{result prefix}XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " + +"will be added as an attribute, prefixed with \"{result prefix}\" For example," + +"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"{result prefix}timestamp\""), + +@WritesAttribute(attribute = "ExtractGrok.exception", description = "if an error occurs, an exception will be written to this attribute, " + +"and the flow routed to 'unmatched' ") +}) public class ExtractGrok extends AbstractProcessor { public static final String FLOWFILE_ATTRIBUTE = "flowfile-attribute"; public static final String FLOWFILE_CONTENT = "flowfile-content"; -private static final String APPLICATION_JSON = "application/json"; - +public static final String APPLICATION_JSON = "application/json"; +public static final String GROK_EXPRESSION_KEY = "Grok Expression"; +public static final String GROK_PATTERN_FILE_KEY = "Grok Pattern file"; +public static final String DESTINATION_KEY = "Destination"; +public static final String CHARACTER_SET_KEY = "Character Set"; +public static final String MAXIMUM_BUFFER_SIZE_KEY = "Maximum Buffer Size"; +public static final String NAMED_CAPTURES_ONLY_KEY = "Named captures only"; +public static final String SINGLE_MATCH_KEY = "Single Match"; +public static final String RESULT_PREFIX_KEY = "result prefix"; +public static final String MATCHED_EXP_ATTR_KEY = "matched expression attribute"; +public static final String EXP_SEPARATOR_KEY = "expression-separator"; +public static final String PATTERN_FILE_LIST_SEPARATOR = ","; + +//properties public static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor.Builder() -.name("Grok Expression") -.description("Grok expression") +.name(GROK_EXPRESSION_KEY) --- End diff -- Can you help me understand the idea behind pulling the string literal out into a member variable? Since the PropertyDescriptor's name is generally not referenced elsewhere, it's usually a lot cleaner to have the name inlined. ---
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2411#discussion_r164499345 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractGrok.java --- @@ -209,78 +287,94 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try { final byte[] byteBuffer = buffer; -session.read(flowFile, new InputStreamCallback() { -@Override -public void process(InputStream in) throws IOException { -StreamUtils.fillBuffer(in, byteBuffer, false); -} -}); +session.read(flowFile, in -> StreamUtils.fillBuffer(in, byteBuffer, false)); final long len = Math.min(byteBuffer.length, flowFile.getSize()); contentString = new String(byteBuffer, 0, (int) len, charset); } finally { bufferQueue.offer(buffer); } -final Match gm = grok.match(contentString); -gm.captures(); - -if (gm.toMap().isEmpty()) { -session.transfer(flowFile, REL_NO_MATCH); -getLogger().info("Did not match any Grok Expressions for FlowFile {}", new Object[]{flowFile}); -return; -} - -final ObjectMapper objectMapper = new ObjectMapper(); -switch (context.getProperty(DESTINATION).getValue()) { -case FLOWFILE_ATTRIBUTE: -MapgrokResults = new HashMap<>(); -for (Map.Entry entry : gm.toMap().entrySet()) { -if (null != entry.getValue()) { -grokResults.put("grok." + entry.getKey(), entry.getValue().toString()); +try{ +for (Grok grok : grokList) { +final Match gm = grok.match(contentString); +gm.captures(); +final Map localResults = gm.toMap(); +if (!localResults.isEmpty()) { + matchedExpressionList.add(grok.getOriginalGrokPattern()); +results.putAll(localResults); +if (breakOnFirstMatch) { +break; } } +} -flowFile = session.putAllAttributes(flowFile, grokResults); -session.getProvenanceReporter().modifyAttributes(flowFile); -session.transfer(flowFile, REL_MATCH); -getLogger().info("Matched {} Grok Expressions and added attributes to FlowFile {}", new Object[]{grokResults.size(), flowFile}); - -break; -case FLOWFILE_CONTENT: -FlowFile conFlowfile = session.write(flowFile, new StreamCallback() { -@Override -public void process(InputStream in, OutputStream out) throws IOException { - out.write(objectMapper.writeValueAsBytes(gm.toMap())); -} -}); -conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), APPLICATION_JSON); -session.getProvenanceReporter().modifyContent(conFlowfile, "Replaced content with parsed Grok fields and values", stopWatch.getElapsed(TimeUnit.MILLISECONDS)); -session.transfer(conFlowfile, REL_MATCH); +if (results.isEmpty()) { +session.transfer(flowFile, REL_NO_MATCH); +getLogger().debug("Did not match any Grok Expression for FlowFile {}", new Object[]{flowFile}); +return; +} + +String matchedExpressions = StringUtils.join(matchedExpressionList, expressionSeparator); +flowFile = session.putAttribute(flowFile, matchedExpressionAttribute, matchedExpressions); -break; +switch (context.getProperty(DESTINATION).getValue()) { +case FLOWFILE_ATTRIBUTE: +Map grokResults = new HashMap<>(); +for (Map.Entry entry : results.entrySet()) { +if (null != entry.getValue()) { +grokResults.put(resultPrefix + entry.getKey(), entry.getValue().toString()); +} +} +flowFile = session.putAllAttributes(flowFile, grokResults); + session.getProvenanceReporter().modifyAttributes(flowFile); +session.transfer(flowFile, REL_MATCH);
[GitHub] nifi pull request #2411: NIFI-4789 Extract grok multi pattern support
GitHub user charlesporter opened a pull request: https://github.com/apache/nifi/pull/2411 NIFI-4789 Extract grok multi pattern support ExtractGrok Processor supports multiple expressions -separated by comma or by specified delimiter -option to return on first match or to run all expressions in list ralated enhancements: -multiple pattern files support -selectable result attribute prefix (default unchanged from '.grok') implemented by creating multiple kraken-Grok objects. It is possible that performance could be improved by using elasticsearch implementation: https://github.com/elastic/elasticsearch/tree/d6d0c13bd63f28347ebdb0f9364e44921c248b8b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/commoncommon/src/main/java/org/elasticsearch/ingest/common/Grok.java You can merge this pull request into a Git repository by running: $ git pull https://github.com/charlesporter/nifi ExtractGrokMultiPattern Alternatively you can review and apply these changes as the patch at: https://github.com/apache/nifi/pull/2411.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2411 commit 66b072a3d7348cb8dc1d2c6ab52634116b1bf6d7 Author: charlesporterDate: 2018-01-17T04:14:45Z ExtractGrok Supports multiple expressions -separated by comma or specified delimiter -option to return on first match or to run all expressions in list other enhancements -multiple pattern files -selectable result attribute prefix commit dd8bc224c5193c8b6b47fb8babf6cb6c8323eef3 Author: charlesporter Date: 2018-01-17T04:14:45Z ExtractGrok Supports multiple expressions -separated by comma or specified delimiter -option to return on first match or to run all expressions in list other enhancements -multiple pattern files -selectable result attribute prefix commit f289df5613046ad8595ce8b980e7063c59ee8cdb Author: charlesporter Date: 2018-01-17T22:04:29Z remerge ---