From f52e1f2a064b31f87d4165af6075716fa7d55046 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:36:08 -0500 Subject: [PATCH 01/18] NIFI-3497 - Added metadata option Added optional to post additional metadata as new attributed if a match is found in the dictionary. --- .../processors/standard/ScanAttribute.java | 127 +++++++++++++----- 1 file changed, 97 insertions(+), 30 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index cd3d36ad6bc8..4b7023983cc5 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -25,6 +25,7 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -37,6 +38,8 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; @@ -60,6 +63,13 @@ @Tags({"scan", "attributes", "search", "lookup"}) @CapabilityDescription("Scans the specified attributes of FlowFiles, checking to see if any of their values are " + "present within the specified dictionary of terms") +@WritesAttributes({ + @WritesAttribute(attribute = "dictionary.hit.{n}.attribute", description = "The attribute name that had a value hit on the dictionary file."), + @WritesAttribute(attribute = "dictionary.hit.{n}.term", description = "The term that had a hit on the dictionary file."), + @WritesAttribute(attribute = "dictionary.hit.{n}.metadata", description = "The metadata returned from the dictionary file associated with the term hit.") +}) + + public class ScanAttribute extends AbstractProcessor { public static final String MATCH_CRITERIA_ALL = "All Must Match"; @@ -97,13 +107,24 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.createRegexValidator(0, 1, false)) .defaultValue(null) .build(); - + + public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() + .name("Dictionary Entry Metadata Demarcator") + .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue(null) + .build(); + private List properties; private Set relationships; private volatile Pattern dictionaryFilterPattern = null; private volatile Pattern attributePattern = null; - private volatile Set dictionaryTerms = null; + private volatile String dictionaryEntryMetadataDemarcator = null; + private volatile Map dictionaryTerms = null; + private volatile Set attributeNameMatches = null; + private volatile SynchronousFileWatcher fileWatcher = null; public static final Relationship REL_MATCHED = new Relationship.Builder() @@ -122,6 +143,8 @@ protected void init(final ProcessorInitializationContext context) { properties.add(ATTRIBUTE_PATTERN); properties.add(MATCHING_CRITERIA); properties.add(DICTIONARY_FILTER); + properties.add(DICTIONARY_ENTRY_METADATA_DEMARCATOR); + this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); @@ -150,11 +173,19 @@ public void onScheduled(final ProcessContext context) throws IOException { this.dictionaryTerms = createDictionary(context); this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()), new LastModifiedMonitor(), 1000L); + + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); } - private Set createDictionary(final ProcessContext context) throws IOException { - final Set terms = new HashSet<>(); + private Map createDictionary(final ProcessContext context) throws IOException { + final Map termsMeta = new HashMap(); + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); + String[] termMeta; + String term; + String meta; + + final File file = new File(context.getProperty(DICTIONARY_FILE).getValue()); try (final InputStream fis = new FileInputStream(file); final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) { @@ -164,10 +195,22 @@ private Set createDictionary(final ProcessContext context) throws IOExce if (line.trim().isEmpty()) { continue; } - - String matchingTerm = line; + + if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) + { + termMeta = line.split(dictionaryEntryMetadataDemarcator); + term = termMeta[0]; + meta = termMeta[1]; + } + else + { + term=line; + meta=""; + } + + String matchingTerm = term; if (dictionaryFilterPattern != null) { - final Matcher matcher = dictionaryFilterPattern.matcher(line); + final Matcher matcher = dictionaryFilterPattern.matcher(term); if (!matcher.matches()) { continue; } @@ -177,20 +220,18 @@ private Set createDictionary(final ProcessContext context) throws IOExce if (matcher.groupCount() == 1) { matchingTerm = matcher.group(1); } else { - matchingTerm = line; + matchingTerm = term; } } - - terms.add(matchingTerm); + termsMeta.put(matchingTerm, meta); } } - - return Collections.unmodifiableSet(terms); + return Collections.unmodifiableMap(termsMeta); } @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { - final List flowFiles = session.get(50); + List flowFiles = session.get(50); if (flowFiles.isEmpty()) { return; } @@ -206,36 +247,62 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final boolean matchAll = context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL); - for (final FlowFile flowFile : flowFiles) { - final boolean matched = matchAll ? allMatch(flowFile, attributePattern, dictionaryTerms) : anyMatch(flowFile, attributePattern, dictionaryTerms); - final Relationship relationship = matched ? REL_MATCHED : REL_UNMATCHED; + for (FlowFile flowFile : flowFiles) { + final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); + flowFile = session.putAllAttributes(flowFile, matched); + + final Relationship relationship = (((matched.size() == (attributeNameMatches.size() * 3) && matchAll) || (matched.size() > 0 && !matchAll))) ? REL_MATCHED : REL_UNMATCHED; session.getProvenanceReporter().route(flowFile, relationship); session.transfer(flowFile, relationship); logger.info("Transferred {} to {}", new Object[]{flowFile, relationship}); } } - private boolean allMatch(final FlowFile flowFile, final Pattern attributePattern, final Set dictionary) { - for (final Map.Entry entry : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(entry.getKey()).matches()) { - if (!dictionary.contains(entry.getValue())) { - return false; + private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; + + for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { + if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { + attributeNameMatches.add(attribute.getKey()); + + if (dictionary.containsKey(attribute.getValue())) { + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); } } } - - return true; + return dictionaryTermMatches; } + + private Map matchAll(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; + + for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { + if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { + attributeNameMatches.add(attribute.getKey()); - private boolean anyMatch(final FlowFile flowFile, final Pattern attributePattern, final Set dictionary) { - for (final Map.Entry entry : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(entry.getKey()).matches()) { - if (dictionary.contains(entry.getValue())) { - return true; + if (dictionary.containsKey(attribute.getValue())) { + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + } + else + { + //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. + dictionaryTermMatches.clear(); + break; } } } - - return false; + return dictionaryTermMatches; } } From 8eb54a50193897cf564eb7d222aae35481168af4 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:46:13 -0500 Subject: [PATCH 02/18] NIFI-3497 test cases for metadata updates Adding test cases to support metadata option update. --- .../standard/TestScanAttribute.java | 188 ++++++++++++++++++ 1 file changed, 188 insertions(+) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java index b4a41369f2b9..53aecf357cb9 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java @@ -16,9 +16,12 @@ */ package org.apache.nifi.processors.standard; +import static org.junit.Assert.*; + import java.util.HashMap; import java.util.Map; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; @@ -145,4 +148,189 @@ public void testWithDictionaryFilter() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); runner.clearTransferState(); } + + @Test + public void testSingleMatchWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.clearTransferState(); + + attributes.remove("produce_name"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "cherry"); + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"cherry"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red fruit"); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "c.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + } + + @Test + public void testAllMatchWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); + runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.clearTransferState(); + + attributes.remove("produce_name"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "banana"); + attributes.put("produce_name_2", "french fries"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "corn"); + attributes.put("produce_name_2", "banana"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.2.term") ,"corn"); + assertEquals(f.getAttribute("dictionary.hit.2.metadata"), "yellow vegetable"); + } + + @Test + public void testWithEmptyEntriesWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", ""); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + } + + @Test + public void testWithDictionaryFilterWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + runner.clearTransferState(); + + attributes.put("produce_name", "tomatoe"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)<.*>"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"tomatoe"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red vegetable"); + + + runner.clearTransferState(); + } } From a7a7b6ace80380416c342809ce95a4f4087bb066 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:48:10 -0500 Subject: [PATCH 03/18] NIFI-3497 - New dictionary files for test Adding new dictionary files to support metadata dictionary option. --- .../ScanAttribute/dictionary-with-empty-new-lines_meta | 8 ++++++++ .../ScanAttribute/dictionary-with-extra-info_meta | 9 +++++++++ .../src/test/resources/ScanAttribute/dictionary1_meta | 5 +++++ 3 files changed, 22 insertions(+) create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta new file mode 100644 index 000000000000..4fbc6b02844f --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta @@ -0,0 +1,8 @@ +banana:yellow fruit + + +zucchini:green vegetable + + + + diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta new file mode 100644 index 000000000000..46c62878b504 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta @@ -0,0 +1,9 @@ +banana:yellow fruit +zucchini:green vegetable +apple:red fruit +lime:green fruit +corn:yellow vegetable +celery:green vegetable +eggplant:purple vegetable +tomatoe:red vegetable +cherry:red fruit \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta new file mode 100644 index 000000000000..bc8716786fb7 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta @@ -0,0 +1,5 @@ +banana:yellow fruit +zucchini:green vegetable +lime:green fruit +corn:yellow vegetable +cherry:red fruit \ No newline at end of file From d71426037b142da8ca04dae38952c164d1614806 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Thu, 23 Feb 2017 10:19:01 -0500 Subject: [PATCH 04/18] NIFI-3497 - excluding test files Adding new test data files to exclude list. --- .../nifi-standard-bundle/nifi-standard-processors/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 5fd9ae24da14..bd06632168cb 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -291,6 +291,9 @@ language governing permissions and limitations under the License. --> src/test/resources/ScanAttribute/dictionary-with-empty-new-lines src/test/resources/ScanAttribute/dictionary-with-extra-info src/test/resources/ScanAttribute/dictionary1 + src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta + src/test/resources/ScanAttribute/dictionary-with-extra-info_meta + src/test/resources/ScanAttribute/dictionary1_meta src/test/resources/TestEncryptContent/text.txt src/test/resources/TestEncryptContent/text.txt.asc src/test/resources/TestIdentifyMimeType/1.txt From 89ec68d14bb34cbe65ff9a4d50ff5321fd4ec0ef Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Fri, 24 Feb 2017 18:09:36 -0500 Subject: [PATCH 05/18] NIFI-3497 - fixing Pcontrib issues --- .../processors/standard/ScanAttribute.java | 92 +++++++++---------- .../standard/TestScanAttribute.java | 52 +++++------ 2 files changed, 69 insertions(+), 75 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index 4b7023983cc5..f1d06a6a2691 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -107,7 +107,6 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.createRegexValidator(0, 1, false)) .defaultValue(null) .build(); - public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() .name("Dictionary Entry Metadata Demarcator") .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") @@ -115,14 +114,14 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .defaultValue(null) .build(); - + private List properties; private Set relationships; private volatile Pattern dictionaryFilterPattern = null; private volatile Pattern attributePattern = null; private volatile String dictionaryEntryMetadataDemarcator = null; - private volatile Map dictionaryTerms = null; + private volatile Map dictionaryTerms = null; private volatile Set attributeNameMatches = null; private volatile SynchronousFileWatcher fileWatcher = null; @@ -144,7 +143,7 @@ protected void init(final ProcessorInitializationContext context) { properties.add(MATCHING_CRITERIA); properties.add(DICTIONARY_FILTER); properties.add(DICTIONARY_ENTRY_METADATA_DEMARCATOR); - + this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); @@ -173,7 +172,7 @@ public void onScheduled(final ProcessContext context) throws IOException { this.dictionaryTerms = createDictionary(context); this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()), new LastModifiedMonitor(), 1000L); - + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); } @@ -184,8 +183,8 @@ private Map createDictionary(final ProcessContext context) throws String[] termMeta; String term; String meta; - - + + final File file = new File(context.getProperty(DICTIONARY_FILE).getValue()); try (final InputStream fis = new FileInputStream(file); final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) { @@ -195,19 +194,16 @@ private Map createDictionary(final ProcessContext context) throws if (line.trim().isEmpty()) { continue; } - - if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) - { - termMeta = line.split(dictionaryEntryMetadataDemarcator); - term = termMeta[0]; - meta = termMeta[1]; - } - else - { - term=line; - meta=""; + + if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) { + termMeta = line.split(dictionaryEntryMetadataDemarcator); + term = termMeta[0]; + meta = termMeta[1]; + } else { + term=line; + meta=""; } - + String matchingTerm = term; if (dictionaryFilterPattern != null) { final Matcher matcher = dictionaryFilterPattern.matcher(term); @@ -248,9 +244,9 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final boolean matchAll = context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL); for (FlowFile flowFile : flowFiles) { - final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); - flowFile = session.putAllAttributes(flowFile, matched); - + final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); + flowFile = session.putAllAttributes(flowFile, matched); + final Relationship relationship = (((matched.size() == (attributeNameMatches.size() * 3) && matchAll) || (matched.size() > 0 && !matchAll))) ? REL_MATCHED : REL_UNMATCHED; session.getProvenanceReporter().route(flowFile, relationship); session.transfer(flowFile, relationship); @@ -258,48 +254,46 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } } - private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); + private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; - int hitCounter = 0; - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); - + attributeNameMatches.add(attribute.getKey()); + if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); } } } return dictionaryTermMatches; } - + private Map matchAll(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; - int hitCounter = 0; - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); + attributeNameMatches.add(attribute.getKey()); if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); - } - else - { - //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. - dictionaryTermMatches.clear(); - break; + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + } else { + //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. + dictionaryTermMatches.clear(); + break; } } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java index 53aecf357cb9..767b2ec73bd7 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.processors.standard; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import java.util.HashMap; import java.util.Map; @@ -148,12 +148,12 @@ public void testWithDictionaryFilter() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); runner.clearTransferState(); } - + @Test public void testSingleMatchWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); FlowFile f; - + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); @@ -165,19 +165,19 @@ public void testSingleMatchWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.clearTransferState(); attributes.remove("produce_name"); runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); runner.clearTransferState(); @@ -188,10 +188,10 @@ public void testSingleMatchWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"cherry"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red fruit"); runner.clearTransferState(); @@ -208,27 +208,27 @@ public void testSingleMatchWithMeta() { public void testAllMatchWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); FlowFile f; - + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", "banana"); runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.clearTransferState(); attributes.remove("produce_name"); @@ -249,18 +249,18 @@ public void testAllMatchWithMeta() { runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.2.term") ,"corn"); assertEquals(f.getAttribute("dictionary.hit.2.metadata"), "yellow vegetable"); } @@ -270,7 +270,7 @@ public void testWithEmptyEntriesWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", ""); @@ -294,7 +294,7 @@ public void testWithDictionaryFilterWithMeta() { runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info_meta"); runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", "banana"); @@ -303,10 +303,10 @@ public void testWithDictionaryFilterWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); runner.clearTransferState(); @@ -323,13 +323,13 @@ public void testWithDictionaryFilterWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"tomatoe"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red vegetable"); - + runner.clearTransferState(); } From de7e348e62c0f7c5fbd334106878ca6ac46935af Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:36:08 -0500 Subject: [PATCH 06/18] NIFI-3497 - Added metadata option Added optional to post additional metadata as new attributed if a match is found in the dictionary. --- .../processors/standard/ScanAttribute.java | 127 +++++++++++++----- 1 file changed, 97 insertions(+), 30 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index cd3d36ad6bc8..4b7023983cc5 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -25,6 +25,7 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -37,6 +38,8 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; @@ -60,6 +63,13 @@ @Tags({"scan", "attributes", "search", "lookup"}) @CapabilityDescription("Scans the specified attributes of FlowFiles, checking to see if any of their values are " + "present within the specified dictionary of terms") +@WritesAttributes({ + @WritesAttribute(attribute = "dictionary.hit.{n}.attribute", description = "The attribute name that had a value hit on the dictionary file."), + @WritesAttribute(attribute = "dictionary.hit.{n}.term", description = "The term that had a hit on the dictionary file."), + @WritesAttribute(attribute = "dictionary.hit.{n}.metadata", description = "The metadata returned from the dictionary file associated with the term hit.") +}) + + public class ScanAttribute extends AbstractProcessor { public static final String MATCH_CRITERIA_ALL = "All Must Match"; @@ -97,13 +107,24 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.createRegexValidator(0, 1, false)) .defaultValue(null) .build(); - + + public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() + .name("Dictionary Entry Metadata Demarcator") + .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue(null) + .build(); + private List properties; private Set relationships; private volatile Pattern dictionaryFilterPattern = null; private volatile Pattern attributePattern = null; - private volatile Set dictionaryTerms = null; + private volatile String dictionaryEntryMetadataDemarcator = null; + private volatile Map dictionaryTerms = null; + private volatile Set attributeNameMatches = null; + private volatile SynchronousFileWatcher fileWatcher = null; public static final Relationship REL_MATCHED = new Relationship.Builder() @@ -122,6 +143,8 @@ protected void init(final ProcessorInitializationContext context) { properties.add(ATTRIBUTE_PATTERN); properties.add(MATCHING_CRITERIA); properties.add(DICTIONARY_FILTER); + properties.add(DICTIONARY_ENTRY_METADATA_DEMARCATOR); + this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); @@ -150,11 +173,19 @@ public void onScheduled(final ProcessContext context) throws IOException { this.dictionaryTerms = createDictionary(context); this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()), new LastModifiedMonitor(), 1000L); + + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); } - private Set createDictionary(final ProcessContext context) throws IOException { - final Set terms = new HashSet<>(); + private Map createDictionary(final ProcessContext context) throws IOException { + final Map termsMeta = new HashMap(); + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); + String[] termMeta; + String term; + String meta; + + final File file = new File(context.getProperty(DICTIONARY_FILE).getValue()); try (final InputStream fis = new FileInputStream(file); final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) { @@ -164,10 +195,22 @@ private Set createDictionary(final ProcessContext context) throws IOExce if (line.trim().isEmpty()) { continue; } - - String matchingTerm = line; + + if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) + { + termMeta = line.split(dictionaryEntryMetadataDemarcator); + term = termMeta[0]; + meta = termMeta[1]; + } + else + { + term=line; + meta=""; + } + + String matchingTerm = term; if (dictionaryFilterPattern != null) { - final Matcher matcher = dictionaryFilterPattern.matcher(line); + final Matcher matcher = dictionaryFilterPattern.matcher(term); if (!matcher.matches()) { continue; } @@ -177,20 +220,18 @@ private Set createDictionary(final ProcessContext context) throws IOExce if (matcher.groupCount() == 1) { matchingTerm = matcher.group(1); } else { - matchingTerm = line; + matchingTerm = term; } } - - terms.add(matchingTerm); + termsMeta.put(matchingTerm, meta); } } - - return Collections.unmodifiableSet(terms); + return Collections.unmodifiableMap(termsMeta); } @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { - final List flowFiles = session.get(50); + List flowFiles = session.get(50); if (flowFiles.isEmpty()) { return; } @@ -206,36 +247,62 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final boolean matchAll = context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL); - for (final FlowFile flowFile : flowFiles) { - final boolean matched = matchAll ? allMatch(flowFile, attributePattern, dictionaryTerms) : anyMatch(flowFile, attributePattern, dictionaryTerms); - final Relationship relationship = matched ? REL_MATCHED : REL_UNMATCHED; + for (FlowFile flowFile : flowFiles) { + final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); + flowFile = session.putAllAttributes(flowFile, matched); + + final Relationship relationship = (((matched.size() == (attributeNameMatches.size() * 3) && matchAll) || (matched.size() > 0 && !matchAll))) ? REL_MATCHED : REL_UNMATCHED; session.getProvenanceReporter().route(flowFile, relationship); session.transfer(flowFile, relationship); logger.info("Transferred {} to {}", new Object[]{flowFile, relationship}); } } - private boolean allMatch(final FlowFile flowFile, final Pattern attributePattern, final Set dictionary) { - for (final Map.Entry entry : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(entry.getKey()).matches()) { - if (!dictionary.contains(entry.getValue())) { - return false; + private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; + + for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { + if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { + attributeNameMatches.add(attribute.getKey()); + + if (dictionary.containsKey(attribute.getValue())) { + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); } } } - - return true; + return dictionaryTermMatches; } + + private Map matchAll(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; + + for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { + if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { + attributeNameMatches.add(attribute.getKey()); - private boolean anyMatch(final FlowFile flowFile, final Pattern attributePattern, final Set dictionary) { - for (final Map.Entry entry : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(entry.getKey()).matches()) { - if (dictionary.contains(entry.getValue())) { - return true; + if (dictionary.containsKey(attribute.getValue())) { + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + } + else + { + //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. + dictionaryTermMatches.clear(); + break; } } } - - return false; + return dictionaryTermMatches; } } From e296268f39bf213a9e8eaa8298b26556c6efe278 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:46:13 -0500 Subject: [PATCH 07/18] NIFI-3497 test cases for metadata updates Adding test cases to support metadata option update. --- .../standard/TestScanAttribute.java | 188 ++++++++++++++++++ 1 file changed, 188 insertions(+) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java index b4a41369f2b9..53aecf357cb9 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java @@ -16,9 +16,12 @@ */ package org.apache.nifi.processors.standard; +import static org.junit.Assert.*; + import java.util.HashMap; import java.util.Map; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; @@ -145,4 +148,189 @@ public void testWithDictionaryFilter() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); runner.clearTransferState(); } + + @Test + public void testSingleMatchWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.clearTransferState(); + + attributes.remove("produce_name"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "cherry"); + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"cherry"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red fruit"); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "c.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + } + + @Test + public void testAllMatchWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); + runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.clearTransferState(); + + attributes.remove("produce_name"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "banana"); + attributes.put("produce_name_2", "french fries"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + attributes.put("produce_name", "corn"); + attributes.put("produce_name_2", "banana"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.2.term") ,"corn"); + assertEquals(f.getAttribute("dictionary.hit.2.metadata"), "yellow vegetable"); + } + + @Test + public void testWithEmptyEntriesWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", ""); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + } + + @Test + public void testWithDictionaryFilterWithMeta() { + final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); + FlowFile f; + + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info_meta"); + runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); + runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); + + final Map attributes = new HashMap<>(); + attributes.put("produce_name", "banana"); + + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); + runner.clearTransferState(); + + attributes.put("produce_name", "tomatoe"); + runner.enqueue(new byte[0], attributes); + runner.run(); + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); + runner.clearTransferState(); + + runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)<.*>"); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); + f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); + + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); + + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"tomatoe"); + assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red vegetable"); + + + runner.clearTransferState(); + } } From b1959341138d3b5004979544fcea86ba36fe6ebb Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 22 Feb 2017 16:48:10 -0500 Subject: [PATCH 08/18] NIFI-3497 - New dictionary files for test Adding new dictionary files to support metadata dictionary option. --- .../ScanAttribute/dictionary-with-empty-new-lines_meta | 8 ++++++++ .../ScanAttribute/dictionary-with-extra-info_meta | 9 +++++++++ .../src/test/resources/ScanAttribute/dictionary1_meta | 5 +++++ 3 files changed, 22 insertions(+) create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta new file mode 100644 index 000000000000..4fbc6b02844f --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta @@ -0,0 +1,8 @@ +banana:yellow fruit + + +zucchini:green vegetable + + + + diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta new file mode 100644 index 000000000000..46c62878b504 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary-with-extra-info_meta @@ -0,0 +1,9 @@ +banana:yellow fruit +zucchini:green vegetable +apple:red fruit +lime:green fruit +corn:yellow vegetable +celery:green vegetable +eggplant:purple vegetable +tomatoe:red vegetable +cherry:red fruit \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta new file mode 100644 index 000000000000..bc8716786fb7 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/ScanAttribute/dictionary1_meta @@ -0,0 +1,5 @@ +banana:yellow fruit +zucchini:green vegetable +lime:green fruit +corn:yellow vegetable +cherry:red fruit \ No newline at end of file From c5d52cf6f0f16496d9ef83411770409c0b2f88d4 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Thu, 23 Feb 2017 10:19:01 -0500 Subject: [PATCH 09/18] NIFI-3497 - excluding test files Adding new test data files to exclude list. --- .../nifi-standard-bundle/nifi-standard-processors/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 5fd9ae24da14..bd06632168cb 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -291,6 +291,9 @@ language governing permissions and limitations under the License. --> src/test/resources/ScanAttribute/dictionary-with-empty-new-lines src/test/resources/ScanAttribute/dictionary-with-extra-info src/test/resources/ScanAttribute/dictionary1 + src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta + src/test/resources/ScanAttribute/dictionary-with-extra-info_meta + src/test/resources/ScanAttribute/dictionary1_meta src/test/resources/TestEncryptContent/text.txt src/test/resources/TestEncryptContent/text.txt.asc src/test/resources/TestIdentifyMimeType/1.txt From 65ed46de9a00518cfe06ecd69bd4f3bbd8d3e662 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Fri, 24 Feb 2017 18:09:36 -0500 Subject: [PATCH 10/18] NIFI-3497 - fixing Pcontrib issues --- .../processors/standard/ScanAttribute.java | 92 +++++++++---------- .../standard/TestScanAttribute.java | 52 +++++------ 2 files changed, 69 insertions(+), 75 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index 4b7023983cc5..f1d06a6a2691 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -107,7 +107,6 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.createRegexValidator(0, 1, false)) .defaultValue(null) .build(); - public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() .name("Dictionary Entry Metadata Demarcator") .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") @@ -115,14 +114,14 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .defaultValue(null) .build(); - + private List properties; private Set relationships; private volatile Pattern dictionaryFilterPattern = null; private volatile Pattern attributePattern = null; private volatile String dictionaryEntryMetadataDemarcator = null; - private volatile Map dictionaryTerms = null; + private volatile Map dictionaryTerms = null; private volatile Set attributeNameMatches = null; private volatile SynchronousFileWatcher fileWatcher = null; @@ -144,7 +143,7 @@ protected void init(final ProcessorInitializationContext context) { properties.add(MATCHING_CRITERIA); properties.add(DICTIONARY_FILTER); properties.add(DICTIONARY_ENTRY_METADATA_DEMARCATOR); - + this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); @@ -173,7 +172,7 @@ public void onScheduled(final ProcessContext context) throws IOException { this.dictionaryTerms = createDictionary(context); this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()), new LastModifiedMonitor(), 1000L); - + this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); } @@ -184,8 +183,8 @@ private Map createDictionary(final ProcessContext context) throws String[] termMeta; String term; String meta; - - + + final File file = new File(context.getProperty(DICTIONARY_FILE).getValue()); try (final InputStream fis = new FileInputStream(file); final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) { @@ -195,19 +194,16 @@ private Map createDictionary(final ProcessContext context) throws if (line.trim().isEmpty()) { continue; } - - if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) - { - termMeta = line.split(dictionaryEntryMetadataDemarcator); - term = termMeta[0]; - meta = termMeta[1]; - } - else - { - term=line; - meta=""; + + if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) { + termMeta = line.split(dictionaryEntryMetadataDemarcator); + term = termMeta[0]; + meta = termMeta[1]; + } else { + term=line; + meta=""; } - + String matchingTerm = term; if (dictionaryFilterPattern != null) { final Matcher matcher = dictionaryFilterPattern.matcher(term); @@ -248,9 +244,9 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final boolean matchAll = context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL); for (FlowFile flowFile : flowFiles) { - final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); - flowFile = session.putAllAttributes(flowFile, matched); - + final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); + flowFile = session.putAllAttributes(flowFile, matched); + final Relationship relationship = (((matched.size() == (attributeNameMatches.size() * 3) && matchAll) || (matched.size() > 0 && !matchAll))) ? REL_MATCHED : REL_UNMATCHED; session.getProvenanceReporter().route(flowFile, relationship); session.transfer(flowFile, relationship); @@ -258,48 +254,46 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } } - private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); + private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; - int hitCounter = 0; - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); - + attributeNameMatches.add(attribute.getKey()); + if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); } } } return dictionaryTermMatches; } - + private Map matchAll(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); + Map dictionaryTermMatches = new HashMap(); + attributeNameMatches = new HashSet(); + + int hitCounter = 0; - int hitCounter = 0; - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); + attributeNameMatches.add(attribute.getKey()); if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); - } - else - { - //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. - dictionaryTermMatches.clear(); - break; + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + } else { + //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. + dictionaryTermMatches.clear(); + break; } } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java index 53aecf357cb9..767b2ec73bd7 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.processors.standard; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import java.util.HashMap; import java.util.Map; @@ -148,12 +148,12 @@ public void testWithDictionaryFilter() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); runner.clearTransferState(); } - + @Test public void testSingleMatchWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); FlowFile f; - + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); @@ -165,19 +165,19 @@ public void testSingleMatchWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.clearTransferState(); attributes.remove("produce_name"); runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); runner.clearTransferState(); @@ -188,10 +188,10 @@ public void testSingleMatchWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"cherry"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red fruit"); runner.clearTransferState(); @@ -208,27 +208,27 @@ public void testSingleMatchWithMeta() { public void testAllMatchWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); FlowFile f; - + runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", "banana"); runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.clearTransferState(); attributes.remove("produce_name"); @@ -249,18 +249,18 @@ public void testAllMatchWithMeta() { runner.enqueue(new byte[0], attributes); runner.run(); - + runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.2.term") ,"corn"); assertEquals(f.getAttribute("dictionary.hit.2.metadata"), "yellow vegetable"); } @@ -270,7 +270,7 @@ public void testWithEmptyEntriesWithMeta() { final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", ""); @@ -294,7 +294,7 @@ public void testWithDictionaryFilterWithMeta() { runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info_meta"); runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - + final Map attributes = new HashMap<>(); attributes.put("produce_name", "banana"); @@ -303,10 +303,10 @@ public void testWithDictionaryFilterWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); runner.clearTransferState(); @@ -323,13 +323,13 @@ public void testWithDictionaryFilterWithMeta() { runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - + runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - + assertEquals(f.getAttribute("dictionary.hit.1.term") ,"tomatoe"); assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red vegetable"); - + runner.clearTransferState(); } From a974f78e033885455cadd2cbffd7e387cbabc4d7 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Sat, 4 Mar 2017 07:43:02 -0500 Subject: [PATCH 11/18] NIFI-3497 Converted properties to use displayName, changed validator on demarcator property and created a method for duplicate code. --- .../processors/standard/ScanAttribute.java | 39 ++++++++++++------- 1 file changed, 25 insertions(+), 14 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index f1d06a6a2691..5cee4605176e 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -44,6 +44,7 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.AbstractProcessor; @@ -76,7 +77,8 @@ public class ScanAttribute extends AbstractProcessor { public static final String MATCH_CRITERIA_ANY = "At Least 1 Must Match"; public static final PropertyDescriptor MATCHING_CRITERIA = new PropertyDescriptor.Builder() - .name("Match Criteria") + .name("match-criteria") + .displayName("Match Criteria") .description("If set to All Must Match, then FlowFiles will be routed to 'matched' only if all specified " + "attributes' values are found in the dictionary. If set to At Least 1 Must Match, FlowFiles will " + "be routed to 'matched' if any attribute specified is found in the dictionary") @@ -85,20 +87,23 @@ public class ScanAttribute extends AbstractProcessor { .defaultValue(MATCH_CRITERIA_ANY) .build(); public static final PropertyDescriptor ATTRIBUTE_PATTERN = new PropertyDescriptor.Builder() - .name("Attribute Pattern") + .name("attribute-pattern") + .displayName("Attribute Pattern") .description("Regular Expression that specifies the names of attributes whose values will be matched against the terms in the dictionary") .required(true) .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) .defaultValue(".*") .build(); public static final PropertyDescriptor DICTIONARY_FILE = new PropertyDescriptor.Builder() - .name("Dictionary File") + .name("dictionary-file") + .displayName("Dictionary File") .description("A new-line-delimited text file that includes the terms that should trigger a match. Empty lines are ignored.") .required(true) .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) .build(); public static final PropertyDescriptor DICTIONARY_FILTER = new PropertyDescriptor.Builder() - .name("Dictionary Filter Pattern") + .name("dictionary-filter-pattern") + .displayName("Dictionary Filter Pattern") .description("A Regular Expression that will be applied to each line in the dictionary file. If the regular expression does not " + "match the line, the line will not be included in the list of terms to search for. If a Matching Group is specified, only the " + "portion of the term that matches that Matching Group will be used instead of the entire term. If not specified, all terms in " @@ -107,11 +112,15 @@ public class ScanAttribute extends AbstractProcessor { .addValidator(StandardValidators.createRegexValidator(0, 1, false)) .defaultValue(null) .build(); + + private static final Validator characterValidator = new StandardValidators.StringLengthValidator(1, 1); + public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() - .name("Dictionary Entry Metadata Demarcator") + .name("dictionary-entry-metadata-demarcator") + .displayName("Dictionary Entry Metadata Demarcator") .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") .required(false) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .addValidator(characterValidator) .defaultValue(null) .build(); @@ -265,10 +274,7 @@ private Map matchAny(final FlowFile flowFile, final Pattern attri attributeNameMatches.add(attribute.getKey()); if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + hitCounter = setDictionaryTermMatch(dictionary, dictionaryTermMatches, hitCounter, attribute); } } } @@ -286,10 +292,7 @@ private Map matchAll(final FlowFile flowFile, final Pattern attri attributeNameMatches.add(attribute.getKey()); if (dictionary.containsKey(attribute.getValue())) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + hitCounter = setDictionaryTermMatch(dictionary, dictionaryTermMatches, hitCounter, attribute); } else { //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. dictionaryTermMatches.clear(); @@ -299,4 +302,12 @@ private Map matchAll(final FlowFile flowFile, final Pattern attri } return dictionaryTermMatches; } + + private int setDictionaryTermMatch(Map dictionary, Map dictionaryTermMatches, int hitCounter, Map.Entry attribute) { + hitCounter++; + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); + dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); + return hitCounter; + } } From 3177eb1852dbc7e70c250c53d038a5bcc5cfa3c5 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Sat, 4 Mar 2017 08:09:06 -0500 Subject: [PATCH 12/18] NIFI-3497 Converted properties to use displayName, changed validator on demarcator property and created a method for duplicate code. --- .../org/apache/nifi/processors/standard/ScanAttribute.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java index 5cee4605176e..12a69744210d 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java @@ -97,8 +97,8 @@ public class ScanAttribute extends AbstractProcessor { public static final PropertyDescriptor DICTIONARY_FILE = new PropertyDescriptor.Builder() .name("dictionary-file") .displayName("Dictionary File") - .description("A new-line-delimited text file that includes the terms that should trigger a match. Empty lines are ignored.") - .required(true) + .description("A new-line-delimited text file that includes the terms that should trigger a match. Empty lines are ignored. The contents of " + + "the text file are loaded into memory when the processor is scheduled and reloaded when the contents are modified.") .required(true) .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) .build(); public static final PropertyDescriptor DICTIONARY_FILTER = new PropertyDescriptor.Builder() From 9c31e45d3f94bf1e73c87379f315f6559e6a23f4 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Mon, 13 Mar 2017 07:55:19 -0400 Subject: [PATCH 13/18] NIFI-1705 Adding AttributesToCSV processor --- .../processors/standard/AttributesToCSV.java | 280 +++++++++++ .../org.apache.nifi.processor.Processor | 1 + .../standard/TestAttributesToCSV.java | 448 ++++++++++++++++++ 3 files changed, 729 insertions(+) create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java new file mode 100644 index 000000000000..2186109bb720 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.processors.standard; + +import org.apache.commons.lang3.StringEscapeUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + + +import java.io.BufferedOutputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; +import java.util.Map; +import java.util.Collections; +import java.util.stream.Collectors; + +@EventDriven +@SideEffectFree +@SupportsBatching +@Tags({"csv", "attributes", "flowfile"}) +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@CapabilityDescription("Generates a CSV representation of the input FlowFile Attributes. The resulting CSV " + + "can be written to either a newly generated attribute named 'CSVAttributes' or written to the FlowFile as content. " + + "If the attribute value contains a comma, newline or double quote, then the attribute value will be " + + "escaped with double quotes. Any double quote characters in the attribute value are escaped with " + + "another double quote. If the attribute value does not contain a comma, newline or double quote, then the " + + "attribute value is returned unchanged.") +@WritesAttribute(attribute = "CSVAttributes", description = "CSV representation of Attributes") +public class AttributesToCSV extends AbstractProcessor { + + private static final String OUTPUT_NEW_ATTRIBUTE = "flowfile-attribute"; + private static final String OUTPUT_OVERWRITE_CONTENT = "flowfile-content"; + private static final String OUTPUT_ATTRIBUTE_NAME = "CSVAttributes"; + private static final String OUTPUT_SEPARATOR = ","; + private static final String OUTPUT_MIME_TYPE = "text/csv"; + + + public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder() + .name("attribute-list") + .displayName("Attribute List") + .description("Comma separated list of attributes to be included in the resulting CSV. If this value " + + "is left empty then all existing Attributes will be included. This list of attributes is " + + "case sensitive. If an attribute specified in the list is not found it will be emitted " + + "to the resulting CSV with an empty string or null depending on the 'Null Value' property. " + + "If a core attribute is specified in this list " + + "and the 'Include Core Attributes' property is false, the core attribute will be included. The attribute list " + + "ALWAYS wins.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder() + .name("destination") + .displayName("Destination") + .description("Control if CSV value is written as a new flowfile attribute 'CSVAttributes' " + + "or written in the flowfile content. Writing to flowfile content will overwrite any " + + "existing flowfile content.") + .required(true) + .allowableValues(OUTPUT_NEW_ATTRIBUTE, OUTPUT_OVERWRITE_CONTENT) + .defaultValue(OUTPUT_NEW_ATTRIBUTE) + .build(); + + public static final PropertyDescriptor INCLUDE_CORE_ATTRIBUTES = new PropertyDescriptor.Builder() + .name("include-core-userSpecifiedAttributes") + .displayName("Include Core Attributes") + .description("Determines if the FlowFile org.apache.nifi.flowfile.attributes.CoreAttributes, which are " + + "contained in every FlowFile, should be included in the final CSV value generated. The Attribute List property " + + "overrides this setting.") + .required(true) + .allowableValues("true", "false") + .defaultValue("true") + .build(); + + public static final PropertyDescriptor NULL_VALUE_FOR_EMPTY_STRING = new PropertyDescriptor.Builder() + .name(("null-value")) + .displayName("Null Value") + .description("If true a non existing or empty attribute will be 'null' in the resulting CSV. If false an empty " + + "string will be placed in the CSV") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") + .description("Successfully converted attributes to CSV").build(); + public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure") + .description("Failed to convert attributes to CSV").build(); + + private List properties; + private Set relationships; + private volatile Set userSpecifiedAttributes; + private volatile Boolean includeCoreAttributes; + private volatile Set coreAttributes; + private volatile boolean destinationContent; + private volatile boolean nullValForEmptyString; + + @Override + protected void init(final ProcessorInitializationContext context) { + final List properties = new ArrayList<>(); + properties.add(ATTRIBUTES_LIST); + properties.add(DESTINATION); + properties.add(INCLUDE_CORE_ATTRIBUTES); + properties.add(NULL_VALUE_FOR_EMPTY_STRING); + this.properties = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + public Set getRelationships() { + return relationships; + } + + + private Map buildAttributesMapForFlowFile(FlowFile ff, Set userSpecifiedAttributes) { + Map result; + if (!userSpecifiedAttributes.isEmpty()) { + //the user gave a list of attributes + result = new HashMap<>(userSpecifiedAttributes.size()); + for (String attribute : userSpecifiedAttributes) { + String val = ff.getAttribute(attribute); + if (val != null && !val.isEmpty()) { + result.put(attribute, val); + } else { + if (nullValForEmptyString) { + result.put(attribute, "null"); + } else { + result.put(attribute, ""); + } + } + } + } else { + //the user did not give a list of attributes, take all the attributes from the flowfile + Map ffAttributes = ff.getAttributes(); + result = new HashMap<>(ffAttributes.size()); + for (Map.Entry e : ffAttributes.entrySet()) { + result.put(e.getKey(), e.getValue()); + } + } + + //now glue on the core attributes if the user wants them. + if(includeCoreAttributes) { + for (String coreAttribute : coreAttributes) { + String val = ff.getAttribute(coreAttribute); + //make sure this coreAttribute is applicable to this flowfile. + if(ff.getAttributes().containsKey(coreAttribute)) { + if (val != null && !val.isEmpty()) { + result.put(coreAttribute, val); + } else { + if (nullValForEmptyString) { + result.put(coreAttribute, "null"); + } else { + result.put(coreAttribute, ""); + } + } + } + } + } else { + //remove core attributes since the user does not want them, unless they are in the attribute list. Attribute List always wins + for (String coreAttribute : coreAttributes) { + //never override user specified attributes, even if the user has selected to exclude core attributes + if(!userSpecifiedAttributes.contains(coreAttribute)) { + result.remove(coreAttribute); + } + } + } + return result; + } + + private Set attributeListStringToSet(String attributeList) { + //take the user specified attribute list string and convert to list of strings. + Set result = new HashSet<>(); + if (StringUtils.isNotBlank(attributeList)) { + String[] ats = StringUtils.split(attributeList, OUTPUT_SEPARATOR); + if (ats != null) { + for (String str : ats) { + String trim = str.trim(); + result.add(trim); + } + } + } + return result; + } + + @OnScheduled + public void onScheduled(ProcessContext context) { + includeCoreAttributes = context.getProperty(INCLUDE_CORE_ATTRIBUTES).asBoolean(); + coreAttributes = Arrays.stream(CoreAttributes.values()).map(CoreAttributes::key).collect(Collectors.toSet()); + userSpecifiedAttributes = attributeListStringToSet(context.getProperty(ATTRIBUTES_LIST).getValue()); + destinationContent = OUTPUT_OVERWRITE_CONTENT.equals(context.getProperty(DESTINATION).getValue()); + nullValForEmptyString = context.getProperty(NULL_VALUE_FOR_EMPTY_STRING).asBoolean(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final FlowFile original = session.get(); + if (original == null) { + return; + } + + final Map atrList = buildAttributesMapForFlowFile(original, userSpecifiedAttributes); + + //escape attribute values + final StringBuilder sb = new StringBuilder(); + for (final String val : atrList.values()) { + sb.append(StringEscapeUtils.escapeCsv(val)); + sb.append(OUTPUT_SEPARATOR); + } + + //check if the output separator is at the end of the string, if so then remove it + if(sb.length() > 0 && sb.lastIndexOf(OUTPUT_SEPARATOR) == sb.length() -1) { + //remove last separator + sb.deleteCharAt(sb.length() - 1); + } + + try { + if (destinationContent) { + FlowFile conFlowfile = session.write(original, (in, out) -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write(sb.toString().getBytes()); + } + }); + conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), OUTPUT_MIME_TYPE); + session.transfer(conFlowfile, REL_SUCCESS); + } else { + FlowFile atFlowfile = session.putAttribute(original, OUTPUT_ATTRIBUTE_NAME , sb.toString()); + session.transfer(atFlowfile, REL_SUCCESS); + } + } catch (Exception e) { + getLogger().error(e.getMessage()); + session.transfer(original, REL_FAILURE); + } + } +} diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index cfcc85afb3a3..322089999864 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. org.apache.nifi.processors.standard.AttributesToJSON +org.apache.nifi.processors.standard.AttributesToCSV org.apache.nifi.processors.standard.Base64EncodeContent org.apache.nifi.processors.standard.CompressContent org.apache.nifi.processors.standard.ControlRate diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java new file mode 100644 index 000000000000..a74576aa264a --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java @@ -0,0 +1,448 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.processors.standard; + +import com.google.common.base.Splitter; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; + +import java.io.IOException; +import java.util.*; +import java.util.regex.Pattern; + +import static org.junit.Assert.*; + + +public class TestAttributesToCSV { + + private static final String OUTPUT_NEW_ATTRIBUTE = "flowfile-attribute"; + private static final String OUTPUT_OVERWRITE_CONTENT = "flowfile-content"; + private static final String OUTPUT_ATTRIBUTE_NAME = "CSVAttributes"; + private static final String OUTPUT_SEPARATOR = ","; + private static final String OUTPUT_MIME_TYPE = "text/csv"; + private static final String SPLIT_REGEX = OUTPUT_SEPARATOR + "(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)"; + + @Test + public void testAttrListNoCoreNullOffNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testAttrListNoCoreNullOffNewAttrToContent() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + //set the destination of the csv string to be an attribute + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + //use only one attribute, which does not exists, as the list of attributes to convert to csv + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testAttrListNoCoreNullOffTwoNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type,beach-length"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",","); + } + + @Test + public void testAttrListNoCoreNullTwoNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "true"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type,beach-length"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes","null,null"); + } + + @Test + public void testNoAttrListNoCoreNullOffToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + //set the destination of the csv string to be an attribute + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testNoAttrListNoCoreNullToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "true"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + + @Test + public void testNoAttrListCoreNullOffToContent() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final byte[] contentData = testRunner.getContentAsByteArray(flowFile); + + final String contentDataString = new String(contentData, "UTF-8"); + + Set contentValues = new HashSet<>(getStrings(contentDataString)); + + assertEquals(6, contentValues.size()); + + assertTrue(contentValues.contains("Malibu Beach")); + assertTrue(contentValues.contains("\"California, US\"")); + assertTrue(contentValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + assertTrue(contentValues.contains(flowFile.getAttribute("filename"))); + assertTrue(contentValues.contains(flowFile.getAttribute("path"))); + assertTrue(contentValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testNoAttrListCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributeValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(6, csvAttributeValues.size()); + + assertTrue(csvAttributeValues.contains("Malibu Beach")); + assertTrue(csvAttributeValues.contains("\"California, US\"")); + assertTrue(csvAttributeValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("filename"))); + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testNoAttrListNoCoreNullOffToContent() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final byte[] contentData = testRunner.getContentAsByteArray(flowFile); + + final String contentDataString = new String(contentData, "UTF-8"); + Set contentValues = new HashSet<>(getStrings(contentDataString)); + + assertEquals(3, contentValues.size()); + + assertTrue(contentValues.contains("Malibu Beach")); + assertTrue(contentValues.contains("\"California, US\"")); + assertTrue(contentValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + } + + + @Test + public void testAttrListNoCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(3, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + } + + @Test + public void testAttrListCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(6, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testAttrListNoCoreNullOffOverrideCoreByAttrListToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement,uuid"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(4, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + + private List getStrings(String sdata) { + return Splitter.on(Pattern.compile(SPLIT_REGEX)).splitToList(sdata); + } + +} From 8b6dcb6e2e2692855bcc4e76f7a3b4a50b8745a3 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Mon, 13 Mar 2017 07:55:19 -0400 Subject: [PATCH 14/18] NIFI-1705 Adding AttributesToCSV processor --- .../processors/standard/AttributesToCSV.java | 280 +++++++++++ .../org.apache.nifi.processor.Processor | 1 + .../standard/TestAttributesToCSV.java | 448 ++++++++++++++++++ 3 files changed, 729 insertions(+) create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java new file mode 100644 index 000000000000..2186109bb720 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.processors.standard; + +import org.apache.commons.lang3.StringEscapeUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + + +import java.io.BufferedOutputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; +import java.util.Map; +import java.util.Collections; +import java.util.stream.Collectors; + +@EventDriven +@SideEffectFree +@SupportsBatching +@Tags({"csv", "attributes", "flowfile"}) +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@CapabilityDescription("Generates a CSV representation of the input FlowFile Attributes. The resulting CSV " + + "can be written to either a newly generated attribute named 'CSVAttributes' or written to the FlowFile as content. " + + "If the attribute value contains a comma, newline or double quote, then the attribute value will be " + + "escaped with double quotes. Any double quote characters in the attribute value are escaped with " + + "another double quote. If the attribute value does not contain a comma, newline or double quote, then the " + + "attribute value is returned unchanged.") +@WritesAttribute(attribute = "CSVAttributes", description = "CSV representation of Attributes") +public class AttributesToCSV extends AbstractProcessor { + + private static final String OUTPUT_NEW_ATTRIBUTE = "flowfile-attribute"; + private static final String OUTPUT_OVERWRITE_CONTENT = "flowfile-content"; + private static final String OUTPUT_ATTRIBUTE_NAME = "CSVAttributes"; + private static final String OUTPUT_SEPARATOR = ","; + private static final String OUTPUT_MIME_TYPE = "text/csv"; + + + public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder() + .name("attribute-list") + .displayName("Attribute List") + .description("Comma separated list of attributes to be included in the resulting CSV. If this value " + + "is left empty then all existing Attributes will be included. This list of attributes is " + + "case sensitive. If an attribute specified in the list is not found it will be emitted " + + "to the resulting CSV with an empty string or null depending on the 'Null Value' property. " + + "If a core attribute is specified in this list " + + "and the 'Include Core Attributes' property is false, the core attribute will be included. The attribute list " + + "ALWAYS wins.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder() + .name("destination") + .displayName("Destination") + .description("Control if CSV value is written as a new flowfile attribute 'CSVAttributes' " + + "or written in the flowfile content. Writing to flowfile content will overwrite any " + + "existing flowfile content.") + .required(true) + .allowableValues(OUTPUT_NEW_ATTRIBUTE, OUTPUT_OVERWRITE_CONTENT) + .defaultValue(OUTPUT_NEW_ATTRIBUTE) + .build(); + + public static final PropertyDescriptor INCLUDE_CORE_ATTRIBUTES = new PropertyDescriptor.Builder() + .name("include-core-userSpecifiedAttributes") + .displayName("Include Core Attributes") + .description("Determines if the FlowFile org.apache.nifi.flowfile.attributes.CoreAttributes, which are " + + "contained in every FlowFile, should be included in the final CSV value generated. The Attribute List property " + + "overrides this setting.") + .required(true) + .allowableValues("true", "false") + .defaultValue("true") + .build(); + + public static final PropertyDescriptor NULL_VALUE_FOR_EMPTY_STRING = new PropertyDescriptor.Builder() + .name(("null-value")) + .displayName("Null Value") + .description("If true a non existing or empty attribute will be 'null' in the resulting CSV. If false an empty " + + "string will be placed in the CSV") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") + .description("Successfully converted attributes to CSV").build(); + public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure") + .description("Failed to convert attributes to CSV").build(); + + private List properties; + private Set relationships; + private volatile Set userSpecifiedAttributes; + private volatile Boolean includeCoreAttributes; + private volatile Set coreAttributes; + private volatile boolean destinationContent; + private volatile boolean nullValForEmptyString; + + @Override + protected void init(final ProcessorInitializationContext context) { + final List properties = new ArrayList<>(); + properties.add(ATTRIBUTES_LIST); + properties.add(DESTINATION); + properties.add(INCLUDE_CORE_ATTRIBUTES); + properties.add(NULL_VALUE_FOR_EMPTY_STRING); + this.properties = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + public Set getRelationships() { + return relationships; + } + + + private Map buildAttributesMapForFlowFile(FlowFile ff, Set userSpecifiedAttributes) { + Map result; + if (!userSpecifiedAttributes.isEmpty()) { + //the user gave a list of attributes + result = new HashMap<>(userSpecifiedAttributes.size()); + for (String attribute : userSpecifiedAttributes) { + String val = ff.getAttribute(attribute); + if (val != null && !val.isEmpty()) { + result.put(attribute, val); + } else { + if (nullValForEmptyString) { + result.put(attribute, "null"); + } else { + result.put(attribute, ""); + } + } + } + } else { + //the user did not give a list of attributes, take all the attributes from the flowfile + Map ffAttributes = ff.getAttributes(); + result = new HashMap<>(ffAttributes.size()); + for (Map.Entry e : ffAttributes.entrySet()) { + result.put(e.getKey(), e.getValue()); + } + } + + //now glue on the core attributes if the user wants them. + if(includeCoreAttributes) { + for (String coreAttribute : coreAttributes) { + String val = ff.getAttribute(coreAttribute); + //make sure this coreAttribute is applicable to this flowfile. + if(ff.getAttributes().containsKey(coreAttribute)) { + if (val != null && !val.isEmpty()) { + result.put(coreAttribute, val); + } else { + if (nullValForEmptyString) { + result.put(coreAttribute, "null"); + } else { + result.put(coreAttribute, ""); + } + } + } + } + } else { + //remove core attributes since the user does not want them, unless they are in the attribute list. Attribute List always wins + for (String coreAttribute : coreAttributes) { + //never override user specified attributes, even if the user has selected to exclude core attributes + if(!userSpecifiedAttributes.contains(coreAttribute)) { + result.remove(coreAttribute); + } + } + } + return result; + } + + private Set attributeListStringToSet(String attributeList) { + //take the user specified attribute list string and convert to list of strings. + Set result = new HashSet<>(); + if (StringUtils.isNotBlank(attributeList)) { + String[] ats = StringUtils.split(attributeList, OUTPUT_SEPARATOR); + if (ats != null) { + for (String str : ats) { + String trim = str.trim(); + result.add(trim); + } + } + } + return result; + } + + @OnScheduled + public void onScheduled(ProcessContext context) { + includeCoreAttributes = context.getProperty(INCLUDE_CORE_ATTRIBUTES).asBoolean(); + coreAttributes = Arrays.stream(CoreAttributes.values()).map(CoreAttributes::key).collect(Collectors.toSet()); + userSpecifiedAttributes = attributeListStringToSet(context.getProperty(ATTRIBUTES_LIST).getValue()); + destinationContent = OUTPUT_OVERWRITE_CONTENT.equals(context.getProperty(DESTINATION).getValue()); + nullValForEmptyString = context.getProperty(NULL_VALUE_FOR_EMPTY_STRING).asBoolean(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final FlowFile original = session.get(); + if (original == null) { + return; + } + + final Map atrList = buildAttributesMapForFlowFile(original, userSpecifiedAttributes); + + //escape attribute values + final StringBuilder sb = new StringBuilder(); + for (final String val : atrList.values()) { + sb.append(StringEscapeUtils.escapeCsv(val)); + sb.append(OUTPUT_SEPARATOR); + } + + //check if the output separator is at the end of the string, if so then remove it + if(sb.length() > 0 && sb.lastIndexOf(OUTPUT_SEPARATOR) == sb.length() -1) { + //remove last separator + sb.deleteCharAt(sb.length() - 1); + } + + try { + if (destinationContent) { + FlowFile conFlowfile = session.write(original, (in, out) -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write(sb.toString().getBytes()); + } + }); + conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), OUTPUT_MIME_TYPE); + session.transfer(conFlowfile, REL_SUCCESS); + } else { + FlowFile atFlowfile = session.putAttribute(original, OUTPUT_ATTRIBUTE_NAME , sb.toString()); + session.transfer(atFlowfile, REL_SUCCESS); + } + } catch (Exception e) { + getLogger().error(e.getMessage()); + session.transfer(original, REL_FAILURE); + } + } +} diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index a93b37fd5bdf..8d4663ca017f 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. org.apache.nifi.processors.standard.AttributesToJSON +org.apache.nifi.processors.standard.AttributesToCSV org.apache.nifi.processors.standard.Base64EncodeContent org.apache.nifi.processors.standard.CompressContent org.apache.nifi.processors.standard.ControlRate diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java new file mode 100644 index 000000000000..a74576aa264a --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java @@ -0,0 +1,448 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.processors.standard; + +import com.google.common.base.Splitter; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; + +import java.io.IOException; +import java.util.*; +import java.util.regex.Pattern; + +import static org.junit.Assert.*; + + +public class TestAttributesToCSV { + + private static final String OUTPUT_NEW_ATTRIBUTE = "flowfile-attribute"; + private static final String OUTPUT_OVERWRITE_CONTENT = "flowfile-content"; + private static final String OUTPUT_ATTRIBUTE_NAME = "CSVAttributes"; + private static final String OUTPUT_SEPARATOR = ","; + private static final String OUTPUT_MIME_TYPE = "text/csv"; + private static final String SPLIT_REGEX = OUTPUT_SEPARATOR + "(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)"; + + @Test + public void testAttrListNoCoreNullOffNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testAttrListNoCoreNullOffNewAttrToContent() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + //set the destination of the csv string to be an attribute + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + //use only one attribute, which does not exists, as the list of attributes to convert to csv + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testAttrListNoCoreNullOffTwoNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type,beach-length"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",","); + } + + @Test + public void testAttrListNoCoreNullTwoNewAttrToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "true"); + + final String NON_PRESENT_ATTRIBUTE_KEY = "beach-type,beach-length"; + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, NON_PRESENT_ATTRIBUTE_KEY); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes","null,null"); + } + + @Test + public void testNoAttrListNoCoreNullOffToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + //set the destination of the csv string to be an attribute + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + @Test + public void testNoAttrListNoCoreNullToAttribute() throws Exception { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "true"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + testRunner.enqueue(ff); + testRunner.run(); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS).get(0) + .assertAttributeExists("CSVAttributes"); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + + testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS) + .get(0).assertAttributeEquals("CSVAttributes",""); + } + + + @Test + public void testNoAttrListCoreNullOffToContent() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final byte[] contentData = testRunner.getContentAsByteArray(flowFile); + + final String contentDataString = new String(contentData, "UTF-8"); + + Set contentValues = new HashSet<>(getStrings(contentDataString)); + + assertEquals(6, contentValues.size()); + + assertTrue(contentValues.contains("Malibu Beach")); + assertTrue(contentValues.contains("\"California, US\"")); + assertTrue(contentValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + assertTrue(contentValues.contains(flowFile.getAttribute("filename"))); + assertTrue(contentValues.contains(flowFile.getAttribute("path"))); + assertTrue(contentValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testNoAttrListCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributeValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(6, csvAttributeValues.size()); + + assertTrue(csvAttributeValues.contains("Malibu Beach")); + assertTrue(csvAttributeValues.contains("\"California, US\"")); + assertTrue(csvAttributeValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("filename"))); + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributeValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testNoAttrListNoCoreNullOffToContent() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_OVERWRITE_CONTENT); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertEquals(OUTPUT_MIME_TYPE, flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final byte[] contentData = testRunner.getContentAsByteArray(flowFile); + + final String contentDataString = new String(contentData, "UTF-8"); + Set contentValues = new HashSet<>(getStrings(contentDataString)); + + assertEquals(3, contentValues.size()); + + assertTrue(contentValues.contains("Malibu Beach")); + assertTrue(contentValues.contains("\"California, US\"")); + assertTrue(contentValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + } + + + @Test + public void testAttrListNoCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(3, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + } + + @Test + public void testAttrListCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(6, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testAttrListNoCoreNullOffOverrideCoreByAttrListToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "beach-name,beach-location,beach-endorsement,uuid"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(4, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + + private List getStrings(String sdata) { + return Splitter.on(Pattern.compile(SPLIT_REGEX)).splitToList(sdata); + } + +} From 0683538c321cbb56934921434dfc5e4507e2b898 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 16 May 2018 11:07:23 -0400 Subject: [PATCH 15/18] Squashed commit of the following: commit 9c31e45d3f94bf1e73c87379f315f6559e6a23f4 Author: Joe Trite Date: Mon Mar 13 07:55:19 2017 -0400 NIFI-1705 Adding AttributesToCSV processor commit 5e9afa9ccf4276c7a2318b761ded77fd514a60d9 Merge: 3177eb185 74cbfc4b6 Author: Joe Trite Date: Sat Mar 4 08:12:39 2017 -0500 Merge remote-tracking branch 'origin/master' # Conflicts: # nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java commit 3177eb1852dbc7e70c250c53d038a5bcc5cfa3c5 Author: Joe Trite Date: Sat Mar 4 08:09:06 2017 -0500 NIFI-3497 Converted properties to use displayName, changed validator on demarcator property and created a method for duplicate code. commit 74cbfc4b69ffde1ddb4442b91e160b66ea3c8b6b Merge: a974f78e0 f8cad0f8c Author: Joe Trite Date: Sat Mar 4 07:47:46 2017 -0500 Merge branch 'master' into master commit a974f78e033885455cadd2cbffd7e387cbabc4d7 Author: Joe Trite Date: Sat Mar 4 07:43:02 2017 -0500 NIFI-3497 Converted properties to use displayName, changed validator on demarcator property and created a method for duplicate code. commit 1bfaef8e87eaa6af84b1a80cb680e0a15eef2f8d Merge: 65ed46de9 89ec68d14 Author: Joe Trite Date: Fri Mar 3 08:01:59 2017 -0500 Merge branch 'master' of https://github.com/joetrite/nifi commit 65ed46de9a00518cfe06ecd69bd4f3bbd8d3e662 Author: Joe Trite Date: Fri Feb 24 18:09:36 2017 -0500 NIFI-3497 - fixing Pcontrib issues commit c5d52cf6f0f16496d9ef83411770409c0b2f88d4 Author: Joe Trite Date: Thu Feb 23 10:19:01 2017 -0500 NIFI-3497 - excluding test files Adding new test data files to exclude list. commit b1959341138d3b5004979544fcea86ba36fe6ebb Author: Joe Trite Date: Wed Feb 22 16:48:10 2017 -0500 NIFI-3497 - New dictionary files for test Adding new dictionary files to support metadata dictionary option. commit e296268f39bf213a9e8eaa8298b26556c6efe278 Author: Joe Trite Date: Wed Feb 22 16:46:13 2017 -0500 NIFI-3497 test cases for metadata updates Adding test cases to support metadata option update. commit de7e348e62c0f7c5fbd334106878ca6ac46935af Author: Joe Trite Date: Wed Feb 22 16:36:08 2017 -0500 NIFI-3497 - Added metadata option Added optional to post additional metadata as new attributed if a match is found in the dictionary. commit 89ec68d14bb34cbe65ff9a4d50ff5321fd4ec0ef Author: Joe Trite Date: Fri Feb 24 18:09:36 2017 -0500 NIFI-3497 - fixing Pcontrib issues commit d71426037b142da8ca04dae38952c164d1614806 Author: Joe Trite Date: Thu Feb 23 10:19:01 2017 -0500 NIFI-3497 - excluding test files Adding new test data files to exclude list. commit a7a7b6ace80380416c342809ce95a4f4087bb066 Author: Joe Trite Date: Wed Feb 22 16:48:10 2017 -0500 NIFI-3497 - New dictionary files for test Adding new dictionary files to support metadata dictionary option. commit 8eb54a50193897cf564eb7d222aae35481168af4 Author: Joe Trite Date: Wed Feb 22 16:46:13 2017 -0500 NIFI-3497 test cases for metadata updates Adding test cases to support metadata option update. commit f52e1f2a064b31f87d4165af6075716fa7d55046 Author: Joe Trite Date: Wed Feb 22 16:36:08 2017 -0500 NIFI-3497 - Added metadata option Added optional to post additional metadata as new attributed if a match is found in the dictionary. --- .../processors/standard/AttributesToCSV.java | 32 +++--- .../standard/TestAttributesToCSV.java | 98 ++++++++++++++++++- 2 files changed, 107 insertions(+), 23 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java index 2186109bb720..0f8d45df5e3b 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AttributesToCSV.java @@ -28,6 +28,7 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.AbstractProcessor; @@ -38,9 +39,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; - -import java.io.BufferedOutputStream; -import java.io.OutputStream; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -77,13 +75,14 @@ public class AttributesToCSV extends AbstractProcessor { .displayName("Attribute List") .description("Comma separated list of attributes to be included in the resulting CSV. If this value " + "is left empty then all existing Attributes will be included. This list of attributes is " + - "case sensitive. If an attribute specified in the list is not found it will be emitted " + + "case sensitive and does not support attribute names that contain commas. If an attribute specified in the list is not found it will be emitted " + "to the resulting CSV with an empty string or null depending on the 'Null Value' property. " + "If a core attribute is specified in this list " + "and the 'Include Core Attributes' property is false, the core attribute will be included. The attribute list " + "ALWAYS wins.") .required(false) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder() @@ -179,18 +178,16 @@ private Map buildAttributesMapForFlowFile(FlowFile ff, Set ffAttributes = ff.getAttributes(); result = new HashMap<>(ffAttributes.size()); - for (Map.Entry e : ffAttributes.entrySet()) { - result.put(e.getKey(), e.getValue()); - } + result.putAll(ffAttributes); } //now glue on the core attributes if the user wants them. if(includeCoreAttributes) { for (String coreAttribute : coreAttributes) { - String val = ff.getAttribute(coreAttribute); //make sure this coreAttribute is applicable to this flowfile. + String val = ff.getAttribute(coreAttribute); if(ff.getAttributes().containsKey(coreAttribute)) { - if (val != null && !val.isEmpty()) { + if (!StringUtils.isEmpty(val)){ result.put(coreAttribute, val); } else { if (nullValForEmptyString) { @@ -232,7 +229,6 @@ private Set attributeListStringToSet(String attributeList) { public void onScheduled(ProcessContext context) { includeCoreAttributes = context.getProperty(INCLUDE_CORE_ATTRIBUTES).asBoolean(); coreAttributes = Arrays.stream(CoreAttributes.values()).map(CoreAttributes::key).collect(Collectors.toSet()); - userSpecifiedAttributes = attributeListStringToSet(context.getProperty(ATTRIBUTES_LIST).getValue()); destinationContent = OUTPUT_OVERWRITE_CONTENT.equals(context.getProperty(DESTINATION).getValue()); nullValForEmptyString = context.getProperty(NULL_VALUE_FOR_EMPTY_STRING).asBoolean(); } @@ -244,27 +240,23 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro return; } + userSpecifiedAttributes = attributeListStringToSet(context.getProperty(ATTRIBUTES_LIST).evaluateAttributeExpressions(original).getValue()); + final Map atrList = buildAttributesMapForFlowFile(original, userSpecifiedAttributes); //escape attribute values + int index = 0; + final int atrListSize = atrList.values().size() -1; final StringBuilder sb = new StringBuilder(); for (final String val : atrList.values()) { sb.append(StringEscapeUtils.escapeCsv(val)); - sb.append(OUTPUT_SEPARATOR); - } - - //check if the output separator is at the end of the string, if so then remove it - if(sb.length() > 0 && sb.lastIndexOf(OUTPUT_SEPARATOR) == sb.length() -1) { - //remove last separator - sb.deleteCharAt(sb.length() - 1); + sb.append(index++ < atrListSize ? OUTPUT_SEPARATOR : ""); } try { if (destinationContent) { FlowFile conFlowfile = session.write(original, (in, out) -> { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write(sb.toString().getBytes()); - } + out.write(sb.toString().getBytes()); }); conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), OUTPUT_MIME_TYPE); session.transfer(conFlowfile, REL_SUCCESS); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java index a74576aa264a..34a0fcb4c3fc 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestAttributesToCSV.java @@ -27,11 +27,14 @@ import org.junit.Test; import java.io.IOException; -import java.util.*; +import java.util.Set; +import java.util.HashSet; +import java.util.List; import java.util.regex.Pattern; -import static org.junit.Assert.*; - +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class TestAttributesToCSV { @@ -441,6 +444,95 @@ public void testAttrListNoCoreNullOffOverrideCoreByAttrListToAttribute() throws assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); } + @Test + public void testAttrListFromExpCoreNullOffToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "true"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "${myAttribs}"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + ff = session.putAttribute(ff, "myAttribs", "beach-name,beach-location,beach-endorsement"); + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(6, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + + @Test + public void testAttrListFromExpNoCoreNullOffOverrideCoreByAttrListToAttribute() throws IOException { + final TestRunner testRunner = TestRunners.newTestRunner(new AttributesToCSV()); + testRunner.setProperty(AttributesToCSV.DESTINATION, OUTPUT_NEW_ATTRIBUTE); + testRunner.setProperty(AttributesToCSV.INCLUDE_CORE_ATTRIBUTES, "false"); + testRunner.setProperty(AttributesToCSV.ATTRIBUTES_LIST, "${myAttribs}"); + testRunner.setProperty(AttributesToCSV.NULL_VALUE_FOR_EMPTY_STRING, "false"); + + ProcessSession session = testRunner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + + ff = session.putAttribute(ff, "beach-name", "Malibu Beach"); + ff = session.putAttribute(ff, "beach-location", "California, US"); + ff = session.putAttribute(ff, "beach-endorsement", "This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim"); + ff = session.putAttribute(ff, "attribute-should-be-eliminated", "This should not be in CSVAttribute!"); + ff = session.putAttribute(ff, "myAttribs", "beach-name,beach-location,beach-endorsement"); + + testRunner.enqueue(ff); + testRunner.run(); + + List flowFilesForRelationship = testRunner.getFlowFilesForRelationship(AttributesToCSV.REL_SUCCESS); + + testRunner.assertTransferCount(AttributesToCSV.REL_FAILURE, 0); + testRunner.assertTransferCount(AttributesToCSV.REL_SUCCESS, 1); + + MockFlowFile flowFile = flowFilesForRelationship.get(0); + + assertNull(flowFile.getAttribute(CoreAttributes.MIME_TYPE.key())); + + final String attributeData = flowFile.getAttribute(OUTPUT_ATTRIBUTE_NAME); + + Set csvAttributesValues = new HashSet<>(getStrings(attributeData)); + + assertEquals(3, csvAttributesValues.size()); + + assertTrue(csvAttributesValues.contains("Malibu Beach")); + assertTrue(csvAttributesValues.contains("\"California, US\"")); + assertTrue(csvAttributesValues.contains("\"This is our family's favorite beach. We highly recommend it. \n\nThanks, Jim\"")); + + + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("filename"))); + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("path"))); + assertTrue(!csvAttributesValues.contains(flowFile.getAttribute("uuid"))); + } + private List getStrings(String sdata) { return Splitter.on(Pattern.compile(SPLIT_REGEX)).splitToList(sdata); } From 48e9909dcfa84fdf0c07673314cb3a2a85d8f887 Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 16 May 2018 13:29:11 -0400 Subject: [PATCH 16/18] Revert "NIFI-3497 - Added metadata option" This reverts commit f52e1f2a064b31f87d4165af6075716fa7d55046. --- .../processors/standard/ScanAttribute.java | 314 ------------------ 1 file changed, 314 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java deleted file mode 100644 index 1c69ee9e8062..000000000000 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanAttribute.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.standard; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.nifi.annotation.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.InputRequirement; -import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.behavior.WritesAttribute; -import org.apache.nifi.annotation.behavior.WritesAttributes; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.Validator; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.AbstractProcessor; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.util.file.monitor.LastModifiedMonitor; -import org.apache.nifi.util.file.monitor.SynchronousFileWatcher; - -@EventDriven -@SideEffectFree -@SupportsBatching -@InputRequirement(Requirement.INPUT_REQUIRED) -@Tags({"scan", "attributes", "search", "lookup"}) -@CapabilityDescription("Scans the specified attributes of FlowFiles, checking to see if any of their values are " - + "present within the specified dictionary of terms") -@WritesAttributes({ - @WritesAttribute(attribute = "dictionary.hit.{n}.attribute", description = "The attribute name that had a value hit on the dictionary file."), - @WritesAttribute(attribute = "dictionary.hit.{n}.term", description = "The term that had a hit on the dictionary file."), - @WritesAttribute(attribute = "dictionary.hit.{n}.metadata", description = "The metadata returned from the dictionary file associated with the term hit.") -}) - - -public class ScanAttribute extends AbstractProcessor { - - public static final String MATCH_CRITERIA_ALL = "All Must Match"; - public static final String MATCH_CRITERIA_ANY = "At Least 1 Must Match"; - - public static final PropertyDescriptor MATCHING_CRITERIA = new PropertyDescriptor.Builder() - .name("match-criteria") - .displayName("Match Criteria") - .description("If set to All Must Match, then FlowFiles will be routed to 'matched' only if all specified " - + "attributes' values are found in the dictionary. If set to At Least 1 Must Match, FlowFiles will " - + "be routed to 'matched' if any attribute specified is found in the dictionary") - .required(true) - .allowableValues(MATCH_CRITERIA_ANY, MATCH_CRITERIA_ALL) - .defaultValue(MATCH_CRITERIA_ANY) - .build(); - public static final PropertyDescriptor ATTRIBUTE_PATTERN = new PropertyDescriptor.Builder() - .name("attribute-pattern") - .displayName("Attribute Pattern") - .description("Regular Expression that specifies the names of attributes whose values will be matched against the terms in the dictionary") - .required(true) - .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) - .defaultValue(".*") - .build(); - public static final PropertyDescriptor DICTIONARY_FILE = new PropertyDescriptor.Builder() - .name("dictionary-file") - .displayName("Dictionary File") - .description("A new-line-delimited text file that includes the terms that should trigger a match. Empty lines are ignored. The contents of " - + "the text file are loaded into memory when the processor is scheduled and reloaded when the contents are modified.") - .required(true) - .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) - .build(); - public static final PropertyDescriptor DICTIONARY_FILTER = new PropertyDescriptor.Builder() - .name("dictionary-filter-pattern") - .displayName("Dictionary Filter Pattern") - .description("A Regular Expression that will be applied to each line in the dictionary file. If the regular expression does not " - + "match the line, the line will not be included in the list of terms to search for. If a Matching Group is specified, only the " - + "portion of the term that matches that Matching Group will be used instead of the entire term. If not specified, all terms in " - + "the dictionary will be used and each term will consist of the text of the entire line in the file") - .required(false) - .addValidator(StandardValidators.createRegexValidator(0, 1, false)) - .defaultValue(null) - .build(); - - private static final Validator characterValidator = new StandardValidators.StringLengthValidator(1, 1); - - public static final PropertyDescriptor DICTIONARY_ENTRY_METADATA_DEMARCATOR = new PropertyDescriptor.Builder() - .name("dictionary-entry-metadata-demarcator") - .displayName("Dictionary Entry Metadata Demarcator") - .description("A single character used to demarcate the dictionary entry string between dictionary value and metadata.") - .required(false) - .addValidator(characterValidator) - .defaultValue(null) - .build(); - - private List properties; - private Set relationships; - - private volatile Pattern dictionaryFilterPattern = null; - private volatile Pattern attributePattern = null; - private volatile String dictionaryEntryMetadataDemarcator = null; - private volatile Map dictionaryTerms = null; - private volatile Set attributeNameMatches = null; - - private volatile SynchronousFileWatcher fileWatcher = null; - - public static final Relationship REL_MATCHED = new Relationship.Builder() - .name("matched") - .description("FlowFiles whose attributes are found in the dictionary will be routed to this relationship") - .build(); - public static final Relationship REL_UNMATCHED = new Relationship.Builder() - .name("unmatched") - .description("FlowFiles whose attributes are not found in the dictionary will be routed to this relationship") - .build(); - - @Override - protected void init(final ProcessorInitializationContext context) { - final List properties = new ArrayList<>(); - properties.add(DICTIONARY_FILE); - properties.add(ATTRIBUTE_PATTERN); - properties.add(MATCHING_CRITERIA); - properties.add(DICTIONARY_FILTER); - properties.add(DICTIONARY_ENTRY_METADATA_DEMARCATOR); - - this.properties = Collections.unmodifiableList(properties); - - final Set relationships = new HashSet<>(); - relationships.add(REL_MATCHED); - relationships.add(REL_UNMATCHED); - this.relationships = Collections.unmodifiableSet(relationships); - } - - @Override - protected List getSupportedPropertyDescriptors() { - return properties; - } - - @Override - public Set getRelationships() { - return relationships; - } - - @OnScheduled - public void onScheduled(final ProcessContext context) throws IOException { - final String filterRegex = context.getProperty(DICTIONARY_FILTER).getValue(); - this.dictionaryFilterPattern = (filterRegex == null) ? null : Pattern.compile(filterRegex); - - final String attributeRegex = context.getProperty(ATTRIBUTE_PATTERN).getValue(); - this.attributePattern = (attributeRegex.equals(".*")) ? null : Pattern.compile(attributeRegex); - - this.dictionaryTerms = createDictionary(context); - this.fileWatcher = new SynchronousFileWatcher(Paths.get(context.getProperty(DICTIONARY_FILE).getValue()), new LastModifiedMonitor(), 1000L); - - this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); - } - - private Map createDictionary(final ProcessContext context) throws IOException { - final Map termsMeta = new HashMap(); - this.dictionaryEntryMetadataDemarcator = context.getProperty(DICTIONARY_ENTRY_METADATA_DEMARCATOR).getValue(); - - String[] termMeta; - String term; - String meta; - - - final File file = new File(context.getProperty(DICTIONARY_FILE).getValue()); - try (final InputStream fis = new FileInputStream(file); - final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) { - - String line; - while ((line = reader.readLine()) != null) { - if (line.trim().isEmpty()) { - continue; - } - - if(dictionaryEntryMetadataDemarcator != null && line.contains(dictionaryEntryMetadataDemarcator)) { - termMeta = line.split(dictionaryEntryMetadataDemarcator); - term = termMeta[0]; - meta = termMeta[1]; - } else { - term=line; - meta=""; - } - - String matchingTerm = term; - if (dictionaryFilterPattern != null) { - final Matcher matcher = dictionaryFilterPattern.matcher(term); - if (!matcher.matches()) { - continue; - } - - // Determine if we should use the entire line or only a part, depending on whether or not - // a Matching Group was specified in the regex. - if (matcher.groupCount() == 1) { - matchingTerm = matcher.group(1); - } else { - matchingTerm = term; - } - } - termsMeta.put(matchingTerm, meta); - } - } - return Collections.unmodifiableMap(termsMeta); - } - - @Override - public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { - List flowFiles = session.get(50); - if (flowFiles.isEmpty()) { - return; - } - - final ComponentLog logger = getLogger(); - try { - if (fileWatcher.checkAndReset()) { - this.dictionaryTerms = createDictionary(context); - } - } catch (final IOException e) { - logger.error("Unable to reload dictionary due to {}", e); - } - - final boolean matchAll = context.getProperty(MATCHING_CRITERIA).getValue().equals(MATCH_CRITERIA_ALL); - - for (FlowFile flowFile : flowFiles) { - final Map matched = (matchAll ? matchAll(flowFile, attributePattern, dictionaryTerms) : matchAny(flowFile, attributePattern, dictionaryTerms)); - flowFile = session.putAllAttributes(flowFile, matched); - - final Relationship relationship = (((matched.size() == (attributeNameMatches.size() * 3) && matchAll) || (matched.size() > 0 && !matchAll))) ? REL_MATCHED : REL_UNMATCHED; - session.getProvenanceReporter().route(flowFile, relationship); - session.transfer(flowFile, relationship); - logger.info("Transferred {} to {}", new Object[]{flowFile, relationship}); - } - } - - private Map matchAny(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); - - int hitCounter = 0; - - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); - - if (dictionary.containsKey(attribute.getValue())) { - hitCounter = setDictionaryTermMatch(dictionary, dictionaryTermMatches, hitCounter, attribute); - } - } - } - return dictionaryTermMatches; - } - - private Map matchAll(final FlowFile flowFile, final Pattern attributePattern, final Map dictionary) { - Map dictionaryTermMatches = new HashMap(); - attributeNameMatches = new HashSet(); - - int hitCounter = 0; - - for (final Map.Entry attribute : flowFile.getAttributes().entrySet()) { - if (attributePattern == null || attributePattern.matcher(attribute.getKey()).matches()) { - attributeNameMatches.add(attribute.getKey()); - - if (dictionary.containsKey(attribute.getValue())) { - hitCounter = setDictionaryTermMatch(dictionary, dictionaryTermMatches, hitCounter, attribute); - } else { - //if one attribute value is not found in the dictionary then no need to continue since this is a matchAll scenario. - dictionaryTermMatches.clear(); - break; - } - } - } - return dictionaryTermMatches; - } - - private int setDictionaryTermMatch(Map dictionary, Map dictionaryTermMatches, int hitCounter, Map.Entry attribute) { - hitCounter++; - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".attribute", attribute.getKey()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".term", attribute.getValue()); - dictionaryTermMatches.put("dictionary.hit." + hitCounter + ".metadata", dictionary.get(attribute.getValue())); - return hitCounter; - } -} From 68feca12029924846c3307b777a7d80ea0785a8f Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 16 May 2018 13:31:55 -0400 Subject: [PATCH 17/18] Revert "NIFI-3497 test cases for metadata updates" This reverts commit 8eb54a50193897cf564eb7d222aae35481168af4. --- .../standard/TestScanAttribute.java | 336 ------------------ 1 file changed, 336 deletions(-) delete mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java deleted file mode 100644 index 767b2ec73bd7..000000000000 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestScanAttribute.java +++ /dev/null @@ -1,336 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.processors.standard; - -import static org.junit.Assert.assertEquals; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; - -import org.junit.Test; - -public class TestScanAttribute { - - @Test - public void testSingleMatch() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1"); - - final Map attributes = new HashMap<>(); - attributes.put("abc", "world"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - attributes.remove("abc"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - attributes.put("abc", "world"); - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "a.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "c.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - } - - @Test - public void testAllMatch() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1"); - runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "a.*"); - - final Map attributes = new HashMap<>(); - attributes.put("abc", "world"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - attributes.remove("abc"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - attributes.put("abc", "world"); - attributes.put("a world", "apart"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - attributes.put("abc", "world"); - attributes.put("a world", "hello"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - } - - @Test - public void testWithEmptyEntries() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines"); - - final Map attributes = new HashMap<>(); - attributes.put("abc", ""); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "a.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - } - - @Test - public void testWithDictionaryFilter() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info"); - runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); - - final Map attributes = new HashMap<>(); - attributes.put("abc", "hello"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - attributes.put("abc", "world"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)<.*>"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - } - - @Test - public void testSingleMatchWithMeta() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - FlowFile f; - - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); - runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - - final Map attributes = new HashMap<>(); - attributes.put("produce_name", "banana"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - - runner.clearTransferState(); - - attributes.remove("produce_name"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - attributes.put("produce_name", "cherry"); - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"cherry"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red fruit"); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "c.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - } - - @Test - public void testAllMatchWithMeta() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - FlowFile f; - - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary1_meta"); - runner.setProperty(ScanAttribute.MATCHING_CRITERIA, ScanAttribute.MATCH_CRITERIA_ALL); - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); - runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - - final Map attributes = new HashMap<>(); - attributes.put("produce_name", "banana"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - - runner.clearTransferState(); - - attributes.remove("produce_name"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - runner.clearTransferState(); - - attributes.put("produce_name", "banana"); - attributes.put("produce_name_2", "french fries"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - attributes.put("produce_name", "corn"); - attributes.put("produce_name_2", "banana"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.2.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.2.term") ,"corn"); - assertEquals(f.getAttribute("dictionary.hit.2.metadata"), "yellow vegetable"); - } - - @Test - public void testWithEmptyEntriesWithMeta() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta"); - runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - - final Map attributes = new HashMap<>(); - attributes.put("produce_name", ""); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.ATTRIBUTE_PATTERN, "pro.*"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - } - - @Test - public void testWithDictionaryFilterWithMeta() { - final TestRunner runner = TestRunners.newTestRunner(new ScanAttribute()); - FlowFile f; - - runner.setProperty(ScanAttribute.DICTIONARY_FILE, "src/test/resources/ScanAttribute/dictionary-with-extra-info_meta"); - runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)"); - runner.setProperty(ScanAttribute.DICTIONARY_ENTRY_METADATA_DEMARCATOR, ":"); - - final Map attributes = new HashMap<>(); - attributes.put("produce_name", "banana"); - - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"banana"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "yellow fruit"); - runner.clearTransferState(); - - attributes.put("produce_name", "tomatoe"); - runner.enqueue(new byte[0], attributes); - runner.run(); - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_UNMATCHED, 1); - runner.clearTransferState(); - - runner.setProperty(ScanAttribute.DICTIONARY_FILTER, "(.*)<.*>"); - runner.enqueue(new byte[0], attributes); - runner.run(); - - runner.assertAllFlowFilesTransferred(ScanAttribute.REL_MATCHED, 1); - f = runner.getFlowFilesForRelationship(ScanAttribute.REL_MATCHED).get(0); - - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.term"); - runner.assertAllFlowFilesContainAttribute("dictionary.hit.1.metadata"); - - assertEquals(f.getAttribute("dictionary.hit.1.term") ,"tomatoe"); - assertEquals(f.getAttribute("dictionary.hit.1.metadata"), "red vegetable"); - - - runner.clearTransferState(); - } -} From 3958f540e68c360603da47681ce8935eea94ef9e Mon Sep 17 00:00:00 2001 From: Joe Trite Date: Wed, 16 May 2018 13:33:51 -0400 Subject: [PATCH 18/18] Revert "NIFI-3497 - excluding test files" This reverts commit d71426037b142da8ca04dae38952c164d1614806. --- .../nifi-standard-bundle/nifi-standard-processors/pom.xml | 3 --- 1 file changed, 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 3f523d8ce3cf..d3fa7f8743fe 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -337,9 +337,6 @@ src/test/resources/ScanAttribute/dictionary-with-empty-new-lines src/test/resources/ScanAttribute/dictionary-with-extra-info src/test/resources/ScanAttribute/dictionary1 - src/test/resources/ScanAttribute/dictionary-with-empty-new-lines_meta - src/test/resources/ScanAttribute/dictionary-with-extra-info_meta - src/test/resources/ScanAttribute/dictionary1_meta src/test/resources/TestEncryptContent/text.txt src/test/resources/TestEncryptContent/text.txt.asc src/test/resources/TestIdentifyMimeType/1.txt