You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2019/10/05 00:45:42 UTC

[nifi] branch master updated: NIFI-6741: Add RecordPath support to scripting components

This is an automated email from the ASF dual-hosted git repository.

alopresto pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new b588073  NIFI-6741: Add RecordPath support to scripting components
b588073 is described below

commit b588073cbbb9655f43b26d854262f449d7de6607
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Fri Oct 4 18:15:16 2019 -0400

    NIFI-6741: Add RecordPath support to scripting components
    
    This closes #3790.
    
    Signed-off-by: Andy LoPresto <al...@apache.org>
---
 .../nifi-scripting-processors/pom.xml              |  11 +++
 .../nifi/processors/script/TestInvokeGroovy.java   |  37 +++++++
 .../test/resources/groovy/test_record_path.groovy  | 110 +++++++++++++++++++++
 3 files changed, 158 insertions(+)

diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
index c69cee5..0483968 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
@@ -53,6 +53,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-path</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-lookup-service-api</artifactId>
         </dependency>
         <dependency>
@@ -108,6 +113,12 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock-record-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.codehaus.groovy</groupId>
             <artifactId>groovy-json</artifactId>
             <version>${scripting.groovy.version}</version>
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java
index 64047ae..4e8c58d 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java
@@ -21,6 +21,8 @@ import org.apache.commons.codec.binary.Hex;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.serialization.record.MockRecordParser;
+import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.MockProcessorInitializationContext;
@@ -35,6 +37,7 @@ import java.security.MessageDigest;
 import java.util.List;
 import java.util.Set;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -209,4 +212,38 @@ public class TestInvokeGroovy extends BaseScriptTest {
         outputFlowFile.assertContentEquals(expectedOutput);
         outputFlowFile.assertAttributeEquals("outAttr", expectedOutput);
     }
+
+    /**
+     * Tests a script that has a Groovy Processor that reads records and outputs a comma-delimited list of fields selected by a given RecordPath expression
+     *
+     * @throws Exception Any error encountered while testing
+     */
+    @Test
+    public void testReadRecordsWithRecordPath() throws Exception {
+        runner.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "Groovy");
+        runner.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/groovy/test_record_path.groovy");
+        runner.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/groovy");
+
+        final MockRecordParser readerService = new MockRecordParser();
+        runner.addControllerService("reader", readerService);
+        runner.enableControllerService(readerService);
+        runner.setProperty("record-reader", "reader");
+        runner.setProperty("record-path", "/age");
+        readerService.addSchemaField("name", RecordFieldType.STRING);
+        readerService.addSchemaField("age", RecordFieldType.INT);
+
+        readerService.addRecord("John Doe", 48);
+        readerService.addRecord("Jane Doe", 47);
+        readerService.addRecord("Jimmy Doe", 14);
+
+        runner.assertValid();
+        runner.enqueue("".getBytes(StandardCharsets.UTF_8));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred("success", 1);
+        final List<MockFlowFile> result = runner.getFlowFilesForRelationship("success");
+        assertEquals(1, result.size());
+        MockFlowFile ff = result.get(0);
+        ff.assertContentEquals("48\n47\n14\n");
+    }
 }
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_path.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_path.groovy
new file mode 100644
index 0000000..74c1429
--- /dev/null
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_record_path.groovy
@@ -0,0 +1,110 @@
+/*
+ * 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.
+ */
+
+import org.apache.nifi.flowfile.FlowFile
+import org.apache.nifi.processor.AbstractProcessor
+import org.apache.nifi.processor.ProcessContext
+import org.apache.nifi.processor.ProcessSession
+import org.apache.nifi.processor.Relationship
+import org.apache.nifi.processor.io.StreamCallback
+import org.apache.nifi.processor.util.StandardValidators
+import org.apache.nifi.record.path.RecordPath
+import org.apache.nifi.record.path.RecordPathResult
+import org.apache.nifi.serialization.*
+import org.apache.nifi.serialization.record.*
+import org.apache.nifi.schema.access.SchemaNotFoundException
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.stream.Collectors
+
+class MyRecordProcessor extends AbstractProcessor {
+
+    // Properties
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for reading incoming data")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .build()
+
+    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
+            .name("record-path")
+            .displayName("Record Path")
+            .description("Specifies the Record Path expression to evaluate against each record")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build()
+
+    def REL_SUCCESS = new Relationship.Builder().name("success").description('FlowFiles that were successfully processed are routed here').build()
+    def REL_FAILURE = new Relationship.Builder().name("failure").description('FlowFiles are routed here if an error occurs during processing').build()
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        def properties = [] as ArrayList
+        properties.add(RECORD_READER)
+        properties.add(RECORD_PATH)
+        properties
+    }
+
+    @Override
+    Set<Relationship> getRelationships() {
+        [REL_SUCCESS, REL_FAILURE] as Set<Relationship>
+    }
+
+    @Override
+    void onTrigger(ProcessContext context, ProcessSession session) {
+        def flowFile = session.get()
+        if (!flowFile) return
+
+        def readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory)
+
+        final Map<String, String> attributes = new HashMap<>()
+        final FlowFile original = flowFile
+        final Map<String, String> originalAttributes = flowFile.attributes
+        final String recordPathExpression = context.getProperty(RECORD_PATH).getValue()
+        final RecordPath recordPath = RecordPath.compile(recordPathExpression)
+        try {
+            flowFile = session.write(flowFile, { inStream, outStream ->
+                def reader = readerFactory.createRecordReader(originalAttributes, inStream, 100, getLogger())
+                try {
+                    Record record
+                    while (record = reader.nextRecord()) {
+                        RecordPathResult result = recordPath.evaluate(record)
+                        def line = result.selectedFields.map({f -> record.getAsString(f.field.fieldName).toString()}).collect(Collectors.joining(',')) + '\n'
+                        outStream.write(line.bytes)
+                    }
+
+                } catch (final SchemaNotFoundException e) {
+                    throw new ProcessException(e.localizedMessage, e)
+                } catch (final MalformedRecordException e) {
+                    throw new ProcessException('Could not parse incoming data', e)
+                } finally {
+                    reader.close()
+                }
+            } as StreamCallback)
+
+        } catch (final Exception e) {
+            getLogger().error('Failed to process {}; will route to failure', [flowFile, e] as Object[])
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+        flowFile = session.putAllAttributes(flowFile, attributes)
+        session.transfer(flowFile, REL_SUCCESS)
+    }
+}
+
+processor = new MyRecordProcessor()
\ No newline at end of file