You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/03/29 17:24:17 UTC

[GitHub] [nifi] simonbence opened a new pull request #4948: NIFI-8273 Adding Scripted Record processors

simonbence opened a new pull request #4948:
URL: https://github.com/apache/nifi/pull/4948


   [NIFI-8273](https://issues.apache.org/jira/browse/NIFI-8273)
   
   This would be my proposal to add some useful feature for working with records: the PR contains three processors, similar to the `ScriptedTransformRecord`. Other than adding an abstract implementation with the common behaviour, the changes should not effect existing features. The new processors:
   - ScriptedValidateRecord: validates records using an user-defined script, and routing them to valid or invalid relationship accordingly.
   - ScriptedFilterRecord: filters out records from incoming flow file based on script
   - ScriptedPartitionRecord: split the incoming records into partitions using a script and then routing them into relationships defined by the user.
   
   Also, [here](https://gist.github.com/simonbence/e2e059b906f635e8c5eb7b5bb8a84341) is a template which might help testing or trying out the processors.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664280132



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       Checking the null explicitly communicates my intention more clearly




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664016266



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       Why do we exactly need this? Do you think it'd worth renaming it to something like _atomicRelationships_?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664551521



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       I understand your point, too. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664003508



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       Optional: I'd replace this with lamdba to be more compact.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664303688



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       Same as above




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663998237



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663996301



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedFilterRecord.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "filter", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+    "This processor provides the ability to filter records out from FlowFiles using the user-provided script. " +
+    "Every record will be evaluated by the script which must return with a boolean value. " +
+    "Records with \"true\" result will be routed to the \"matching\" relationship in a batch. " +
+    "Other records will be filtered out."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedRouteRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedFilterRecord extends ScriptedRouterProcessor<Boolean> {
+    static final Relationship RELATIONSHIP_MATCHING = new Relationship.Builder()
+            .name("matching")
+            .description(
+                "Matching records of the original FlowFile will be routed to this relationship. " +
+                "If there are no matching records, no FlowFile will be routed here."
+            )
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the number of filtered or remaining records.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILED = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency, _RELATIONSHIPS_ may be final.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r667013360



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {

Review comment:
       I see the value in doing this splitting and routing in one step. Both it feels "natural" to send records in different "buckets" into different paths and performance wise this might come handy with bigger amount of records. If you are not against it, I would rather keep the processor here. But I also understand your concerns and think them valid, so I see it as a possible solution to provide a much more exhaustive documentation of the processor's behaviour, with examples (in the additionalDetails). How do you feel about this approach?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664076677



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Could you please remove _false_ from here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] timeabarna commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
timeabarna commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r639552668



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedFilterRecord.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedFilterRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] MATCHING_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] MATCHING_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] NON_MATCHING_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] NON_MATCHING_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNonMatchingRecordsOnly() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] timeabarna commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
timeabarna commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r639581961



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.ListRecordSet;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptEngine scriptEngine = pollScriptEngine();
+        if (scriptEngine == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailedRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptEngine(scriptEngine);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailedRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, List<Record>> outgoingRecords = new HashMap<>();
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!outgoingRecords.containsKey(outgoingRelationship.get())) {
+                                        outgoingRecords.put(outgoingRelationship.get(), new LinkedList<>());
+                                    }
+
+                                    outgoingRecords.get(outgoingRelationship.get()).add(record);
+                                } else {
+                                    getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue));
+                                }
+                            } else {
+                                throw new ProcessException("Script result is not applicable: " + String.valueOf(evaluatedValue));
+                            }
+                        }
+
+                        // Creating and sending outgoing flow files
+                        for (final Map.Entry<Relationship, List<Record>> entry : outgoingRecords.entrySet()) {
+                            final Relationship relationship = entry.getKey();
+                            final List<Record> records = entry.getValue();
+
+                            if (records.isEmpty()) {
+                                getLogger().debug("No outgoing records for relationship {}", relationship.getName());
+                                continue;
+                            }
+
+                            FlowFile outgoingFlowFile = session.create(incomingFlowFile);
+                            final Map<String, String> attributesToAdd = new HashMap<>();
+
+                            try (
+                                final OutputStream out = session.write(outgoingFlowFile);
+                                final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            ) {
+                                attributesToAdd.put("mime.type", writer.getMimeType());
+
+                                if (records.size() == 1) {
+                                    final WriteResult writeResult = writer.write(records.get(0));

Review comment:
       WriteResult can be extracted before if, same in ScriptedTransformRecord class row 231.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEvaluator.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.serialization.record.Record;
+
+import javax.script.ScriptException;
+
+/**
+ * Used by scripted record processors to enclose script engines for different languages.
+ */
+interface ScriptEvaluator {
+
+    /**
+     * Evaluates the enclosed script using the record as arugment. Returns with the script's return value.

Review comment:
       @simonbence Thank you very much for your contribution. I've got some minor stylistic findings worth checking.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEvaluator.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.serialization.record.Record;
+
+import javax.script.ScriptException;
+
+/**
+ * Used by scripted record processors to enclose script engines for different languages.
+ */
+interface ScriptEvaluator {
+
+    /**
+     * Evaluates the enclosed script using the record as arugment. Returns with the script's return value.

Review comment:
       Typo in argument.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664285050



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       I was thinking on this originally but I decided not to do. My key reasons: 1. this behaviour is specific to these two classes and not part of the more generic abstraction `ScriptedProcessor`, thus the abstraction would break. 2. this is not as big amount of code to introduce an intermediate level of abstraction (Note: originally these two classes were not separated)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664011535



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       ```suggestion
                          protected BiFunction<FlowFile, OutputStream, RecordSetWriter> getFlowFileOutputStreamRecordSetWriterBiFunction(RecordSchema schema, RecordSetWriterFactory writerFactory) {
           return (outgoingFlowFile, out) -> {
               try {
                   return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
               } catch (final IOException | SchemaNotFoundException e) {
                   throw new ProcessException("Could not create RecordSetWriter", e);
               }
           };
       }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663994801



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Minor: You can use method references here:
   
   ```suggestion
           final Set<String> attributeNames = attributes.stream().map(Map::keySet).flatMap(Collection::stream).collect(Collectors.toSet());
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#issuecomment-814092452


   Thanks @simonbence ! Will review


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] timeabarna commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
timeabarna commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r639552943



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedFilterRecord.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedFilterRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] MATCHING_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] MATCHING_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] NON_MATCHING_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] NON_MATCHING_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNonMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(NON_MATCHING_RECORD_1);
+        recordReader.addRecord(NON_MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingAndNonMatchingRecords() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(NON_MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+        recordReader.addRecord(NON_MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedValidateRecord.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedValidateRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] VALID_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] VALID_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] INVALID_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] INVALID_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsValidRecordsOnly() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedValidateRecord.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedValidateRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] VALID_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] VALID_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] INVALID_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] INVALID_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsValidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenNoInvalidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsInvalidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(INVALID_RECORD_1);
+        recordReader.addRecord(INVALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenInvalidFlowFileContains(new Object[][]{INVALID_RECORD_1, INVALID_RECORD_2});
+        thenNoValidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsBothValidAndInvalidRecords() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(INVALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+        recordReader.addRecord(INVALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenInvalidFlowFileContains(new Object[][]{INVALID_RECORD_1, INVALID_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsNonMatchingRecord() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedValidateRecord.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedValidateRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] VALID_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] VALID_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] INVALID_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] INVALID_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsValidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenNoInvalidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsInvalidRecordsOnly() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedFilterRecord.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedFilterRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] MATCHING_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] MATCHING_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] NON_MATCHING_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] NON_MATCHING_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNonMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(NON_MATCHING_RECORD_1);
+        recordReader.addRecord(NON_MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingAndNonMatchingRecords() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedValidateRecord.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedValidateRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] VALID_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] VALID_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] INVALID_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] INVALID_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsValidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenNoInvalidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsInvalidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(INVALID_RECORD_1);
+        recordReader.addRecord(INVALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenInvalidFlowFileContains(new Object[][]{INVALID_RECORD_1, INVALID_RECORD_2});
+        thenNoValidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsBothValidAndInvalidRecords() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsOnlyNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedValidateRecord.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedValidateRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] VALID_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] VALID_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] INVALID_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] INVALID_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsValidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenNoInvalidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsInvalidRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(INVALID_RECORD_1);
+        recordReader.addRecord(INVALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenInvalidFlowFileContains(new Object[][]{INVALID_RECORD_1, INVALID_RECORD_2});
+        thenNoValidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsBothValidAndInvalidRecords() throws Exception {
+        // given
+        recordReader.addRecord(VALID_RECORD_1);
+        recordReader.addRecord(INVALID_RECORD_1);
+        recordReader.addRecord(VALID_RECORD_2);
+        recordReader.addRecord(INVALID_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenValidFlowFileContains(new Object[][]{VALID_RECORD_1, VALID_RECORD_2});
+        thenInvalidFlowFileContains(new Object[][]{INVALID_RECORD_1, INVALID_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenNoValidFlowFile();
+        thenNoInvalidFlowFile();
+    }
+
+    @Test
+    public void testIncomingFlowFileCannotBeRead() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedFilterRecord.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedFilterRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] MATCHING_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] MATCHING_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] NON_MATCHING_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] NON_MATCHING_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNonMatchingRecordsOnly() throws Exception {
+        // given
+        recordReader.addRecord(NON_MATCHING_RECORD_1);
+        recordReader.addRecord(NON_MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingAndNonMatchingRecords() throws Exception {
+        // given
+        recordReader.addRecord(MATCHING_RECORD_1);
+        recordReader.addRecord(NON_MATCHING_RECORD_1);
+        recordReader.addRecord(MATCHING_RECORD_2);
+        recordReader.addRecord(NON_MATCHING_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileContains(new Object[][]{MATCHING_RECORD_1, MATCHING_RECORD_2});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenMatchingFlowFileIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileCannotBeRead() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsOnlyNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileCannotBeRead() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedFilterRecord.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestScriptedFilterRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT = "return record.getValue(\"first\") == 1";
+
+    private static final Object[] MATCHING_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] MATCHING_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] NON_MATCHING_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] NON_MATCHING_RECORD_2 = new Object[] {2, "ipsum"};
+
+    @Test
+    public void testIncomingFlowFileContainsMatchingRecordsOnly() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsOnlyNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFileContainsNoRecords() throws Exception {
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFileCannotBeRead() throws Exception {
+        // given
+        recordReader.failAfter(0);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToFailed();
+        thenGivenRouteIsEmpty(RELATIONSHIP_1);
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testMultiplePartitionPointToTheSameRelationship() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_1_RECORD_2);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1, PARTITION_1_RECORD_2});
+        thenGivenRouteIsEmpty(RELATIONSHIP_2);
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForMultipleRoutes() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingIsEmpty();
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsNonMatchingRecord() throws Exception {
+        // given
+        recordReader.addRecord(PARTITION_1_RECORD_1);
+        recordReader.addRecord(PARTITION_2_RECORD_1);
+        recordReader.addRecord(PARTITION_3_RECORD_1);
+
+        // when
+        whenTriggerProcessor();
+
+        // then
+        thenIncomingFlowFileIsRoutedToOriginal();
+        thenGivenRouteContains(RELATIONSHIP_1, new Object[][]{PARTITION_1_RECORD_1});
+        thenGivenRouteContains(RELATIONSHIP_2, new Object[][]{PARTITION_2_RECORD_1});
+        thenNonMatchingContains(new Object[][]{PARTITION_3_RECORD_1});
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsOnlyNonMatchingRecord() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =
+        "if (record.getValue(\"first\") == 1) {\n" +
+        "   return \"partition1\";\n" +
+        "} else if (record.getValue(\"first\") == 2) {\n" +
+        "   return \"partition2\";\n" +
+        "} else {\n" +
+        "   return \"partition3\";\n" +
+        "}\n";
+
+    private static final Object[] PARTITION_1_RECORD_1 = new Object[] {1, "lorem"};
+    private static final Object[] PARTITION_1_RECORD_2 = new Object[] {1, "ipsum"};
+    private static final Object[] PARTITION_2_RECORD_1 = new Object[] {2, "lorem"};
+    private static final Object[] PARTITION_3_RECORD_1 = new Object[] {3, "lorem"};
+    private static final String PARTITION_1 = "partition1";
+    private static final String PARTITION_2 = "partition2";
+    private static final String PARTITION_3 = "partition3";
+    private static final String RELATIONSHIP_1 = "relationship1";
+    private static final String RELATIONSHIP_2 = "relationship2";
+
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        testRunner.setProperty(PARTITION_1, RELATIONSHIP_1);
+        testRunner.setProperty(PARTITION_2, RELATIONSHIP_2);
+    }
+
+    @Test
+    public void testIncomingFlowFilesContainsRecordForOneRoute() throws Exception {

Review comment:
       Exception is never thrown, it can be removed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#issuecomment-858128745


   Will review...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664305254



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {

Review comment:
       As I argued above, splitting the methods of the anonymous implementations would make this part harder to read and because of this I prefer to leave it as it is. (However, I do understand the cyclomatic complexity is high)

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Please see above




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664014241



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r676855320



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {

Review comment:
       I see this in multiple places. If this shouldn't happen, then I think pollScriptRunner() should just check for this and if it does occur, throw ProcessSession. That will result in the session automatically being rolled back and would provide more information in logs about what is happening, rather than silently failing - plus results in cleaner code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664015264



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
+    private final Map<String, Relationship> routes = new ConcurrentHashMap<>();
+
+    public ScriptedRouteRecord() {
+        super(String.class);
+    }
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        relationships.set(new HashSet<>(RELATIONSHIPS));
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships.get();
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .required(false)
+            .name(propertyDescriptorName)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dynamic(true)
+            .build();
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (descriptor.isDynamic()) {
+            final Set<Relationship> relationships = new HashSet<>(this.relationships.get());
+            final String routeName = descriptor.getName();
+
+            if (shouldDeleteDynamicRelationship(routeName, oldValue)) {
+                relationships.remove(new Relationship.Builder().name(oldValue).build());
+            }
+
+            if (newValue == null) {
+                routes.remove(routeName);
+            } else {
+                final Relationship newRelationship = new Relationship.Builder().name(newValue).build();
+                routes.put(routeName, newRelationship);
+                relationships.add(newRelationship);
+            }
+
+            this.relationships.set(relationships);
+        }
+    }
+
+    private boolean shouldDeleteDynamicRelationship(final String routeName, final String oldValue) {
+        // If no further route points to the same relationship and it is not a static relationship, it must be removed
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());
+        return !routes.entrySet().stream().filter(e -> !e.getKey().equals(routeName)).map(e -> e.getValue().getName()).collect(Collectors.toSet()).contains(oldValue)
+            && !staticRelationships.contains(oldValue);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+        final Map<PropertyDescriptor, String> properties = validationContext.getProperties();
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());

Review comment:
       Minor: Here as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664072770



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       Optional: There are some improvement opportunities in this class beyond your modifications, feel free to pass.
   1. Method extraction to reduce cognitive complexity: 158-185
   2. Method extraction to reduce cognitive complexity: 187-213
   3. Removing redundant array creation from the logging method towards the bottom.
   4. To throw more specific exceptions instead of Runtime exceptions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663993537



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;

Review comment:
       Would you please reorder the modifiers to comply with JLS?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Minor: You can use method reference here:
   
   ```suggestion
           final Set<String> attributeNames = attributes.stream().map(Map::keySet).flatMap(Collection::stream).collect(Collectors.toSet());
   ```

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Minor: You can use method references here:
   
   ```suggestion
           final Set<String> attributeNames = attributes.stream().map(Map::keySet).flatMap(Collection::stream).collect(Collectors.toSet());
   ```

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedFilterRecord.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "filter", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+    "This processor provides the ability to filter records out from FlowFiles using the user-provided script. " +
+    "Every record will be evaluated by the script which must return with a boolean value. " +
+    "Records with \"true\" result will be routed to the \"matching\" relationship in a batch. " +
+    "Other records will be filtered out."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedRouteRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedFilterRecord extends ScriptedRouterProcessor<Boolean> {
+    static final Relationship RELATIONSHIP_MATCHING = new Relationship.Builder()
+            .name("matching")
+            .description(
+                "Matching records of the original FlowFile will be routed to this relationship. " +
+                "If there are no matching records, no FlowFile will be routed here."
+            )
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the number of filtered or remaining records.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILED = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency, _RELATIONSHIPS_ may be final.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       _instanceof_ returns _false_ for _nulls_, I think you can remove it.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       What do you think of moving this up as a separate method e.g. _createRecordSetWriterFactory_ to _ScriptedProcessor_? This is duplicated in _ScriptedRouterProcessor_ 142-148

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       Optional: I'd replace this with lamdba to be more compact.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Would you please remove the redundant initializer here?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(

Review comment:
       This method is quite complex. What do you think of extracting the parts at the lines 184-204 and 209-218 into separate methods to increase readability?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       ```suggestion
                          protected BiFunction<FlowFile, OutputStream, RecordSetWriter> getFlowFileOutputStreamRecordSetWriterBiFunction(RecordSchema schema, RecordSetWriterFactory writerFactory) {
           return (outgoingFlowFile, out) -> {
               try {
                   return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
               } catch (final IOException | SchemaNotFoundException e) {
                   throw new ProcessException("Could not create RecordSetWriter", e);
               }
           };
       }
   ```

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedProcessor.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.script;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.script.ScriptingComponentHelper;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+abstract class ScriptedProcessor extends AbstractProcessor implements Searchable {
+    protected static final String PYTHON_SCRIPT_LANGUAGE = "python";
+    protected static final Set<String> SCRIPT_OPTIONS = ScriptingComponentUtils.getAvailableEngines();
+
+    protected volatile String scriptToRun = null;
+    protected final AtomicReference<CompiledScript> compiledScriptRef = new AtomicReference<>();
+    private final ScriptingComponentHelper scriptingComponentHelper = new ScriptingComponentHelper();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+
+    @OnScheduled
+    public void setup(final ProcessContext context) throws IOException {
+        if (!scriptingComponentHelper.isInitialized.get()) {
+            scriptingComponentHelper.createResources(false);
+        }
+
+        scriptingComponentHelper.setupVariables(context);
+        scriptToRun = scriptingComponentHelper.getScriptBody();
+
+        if (scriptToRun == null && scriptingComponentHelper.getScriptPath() != null) {
+            try (final FileInputStream scriptStream = new FileInputStream(scriptingComponentHelper.getScriptPath())) {
+                scriptToRun = IOUtils.toString(scriptStream, Charset.defaultCharset());
+            }
+        }
+
+        // Create a script runner for each possible task
+        final int maxTasks = context.getMaxConcurrentTasks();
+        scriptingComponentHelper.setupScriptRunners(maxTasks, scriptToRun, getLogger());
+
+        // Always compile when first run
+        compiledScriptRef.set(null);
+    }
+
+    protected ScriptEvaluator createEvaluator(final ScriptEngine scriptEngine, final FlowFile flowFile) throws ScriptException {
+        if (PYTHON_SCRIPT_LANGUAGE.equalsIgnoreCase(scriptEngine.getFactory().getLanguageName())) {
+            final CompiledScript compiledScript = getOrCompileScript((Compilable) scriptEngine, scriptToRun);
+            return new PythonScriptEvaluator(scriptEngine, compiledScript, flowFile, getLogger());
+        }
+
+        return new InterpretedScriptEvaluator(scriptEngine, scriptToRun, flowFile, getLogger());
+    }
+
+    private CompiledScript getOrCompileScript(final Compilable scriptEngine, final String scriptToRun) throws ScriptException {
+        final CompiledScript existing = compiledScriptRef.get();
+        if (existing != null) {
+            return existing;
+        }
+
+        final CompiledScript compiled = scriptEngine.compile(scriptToRun);
+        final boolean updated = compiledScriptRef.compareAndSet(null, compiled);
+        if (updated) {
+            return compiled;
+        }
+
+        return compiledScriptRef.get();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        return scriptingComponentHelper.customValidate(validationContext);
+    }
+
+    @Override
+    public Collection<SearchResult> search(final SearchContext context) {
+        return ScriptingComponentUtils.search(context, getLogger());
+    }
+
+    protected static Bindings setupBindings(final ScriptEngine scriptEngine) {
+        Bindings bindings = scriptEngine.getBindings(ScriptContext.ENGINE_SCOPE);
+        if (bindings == null) {
+            bindings = new SimpleBindings();
+        }
+
+        scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
+
+        return bindings;
+    }
+
+    protected ScriptRunner pollScriptRunner() {
+        return scriptingComponentHelper.scriptRunnerQ.poll();
+    }
+
+    protected void offerScriptRunner(ScriptRunner scriptRunner) {
+        scriptingComponentHelper.scriptRunnerQ.offer(scriptRunner);

Review comment:
       Is it possible at any point that we cannot add the _scriptRunner_ to the queue? If so, should the _false_ return value of _offer_ get handled?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
+    private final Map<String, Relationship> routes = new ConcurrentHashMap<>();
+
+    public ScriptedRouteRecord() {
+        super(String.class);
+    }
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        relationships.set(new HashSet<>(RELATIONSHIPS));
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships.get();
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .required(false)
+            .name(propertyDescriptorName)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dynamic(true)
+            .build();
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (descriptor.isDynamic()) {
+            final Set<Relationship> relationships = new HashSet<>(this.relationships.get());
+            final String routeName = descriptor.getName();
+
+            if (shouldDeleteDynamicRelationship(routeName, oldValue)) {
+                relationships.remove(new Relationship.Builder().name(oldValue).build());
+            }
+
+            if (newValue == null) {
+                routes.remove(routeName);
+            } else {
+                final Relationship newRelationship = new Relationship.Builder().name(newValue).build();
+                routes.put(routeName, newRelationship);
+                relationships.add(newRelationship);
+            }
+
+            this.relationships.set(relationships);
+        }
+    }
+
+    private boolean shouldDeleteDynamicRelationship(final String routeName, final String oldValue) {
+        // If no further route points to the same relationship and it is not a static relationship, it must be removed
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());

Review comment:
       Minor: Lambda can be replaced with _Relationship::getName_.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
+    private final Map<String, Relationship> routes = new ConcurrentHashMap<>();
+
+    public ScriptedRouteRecord() {
+        super(String.class);
+    }
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        relationships.set(new HashSet<>(RELATIONSHIPS));
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships.get();
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .required(false)
+            .name(propertyDescriptorName)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dynamic(true)
+            .build();
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (descriptor.isDynamic()) {
+            final Set<Relationship> relationships = new HashSet<>(this.relationships.get());
+            final String routeName = descriptor.getName();
+
+            if (shouldDeleteDynamicRelationship(routeName, oldValue)) {
+                relationships.remove(new Relationship.Builder().name(oldValue).build());
+            }
+
+            if (newValue == null) {
+                routes.remove(routeName);
+            } else {
+                final Relationship newRelationship = new Relationship.Builder().name(newValue).build();
+                routes.put(routeName, newRelationship);
+                relationships.add(newRelationship);
+            }
+
+            this.relationships.set(relationships);
+        }
+    }
+
+    private boolean shouldDeleteDynamicRelationship(final String routeName, final String oldValue) {
+        // If no further route points to the same relationship and it is not a static relationship, it must be removed
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());
+        return !routes.entrySet().stream().filter(e -> !e.getKey().equals(routeName)).map(e -> e.getValue().getName()).collect(Collectors.toSet()).contains(oldValue)
+            && !staticRelationships.contains(oldValue);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final List<ValidationResult> results = new ArrayList<>();
+        final Map<PropertyDescriptor, String> properties = validationContext.getProperties();
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());

Review comment:
       Minor: Here as well.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       Why do we exactly need this? Do you think it'd worth renaming it to something like _atomicRelationships_?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedValidateRecord.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "validate", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+        "This processor provides the ability to validate records in FlowFiles using the user-provided script. " +
+        "The script is expected to have a record as incoming argument and return with a boolean value. " +
+        "Based on this result, the processor categorizes the records as \"valid\" or \"invalid\" and routes them to the respective relationship in batch. " +
+        "Additionally the original FlowFile will be routed to the \"original\" relationship or in case of unsuccessful processing, to the \"failed\" relationship."
+)
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedValidateRecord extends ScriptedRouterProcessor<Boolean> {
+
+    static final Relationship RELATIONSHIP_VALID = new Relationship.Builder()
+            .name("valid")
+            .description(
+                "FlowFile containing the valid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no valid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_INVALID = new Relationship.Builder()
+            .name("invalid")
+            .description(
+                "FlowFile containing the invalid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no invalid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the FlowFiles might routed to \"valid\" and \"invalid\" relationships.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("In case of any issue during processing the incoming flow file, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       Optional: There are some improvement opportunities in this class beyond your modifications, feel free to pass.
   1. Method extraction to reduce cognitive complexity: 158-185
   2. Method extraction to reduce cognitive complexity: 187-213
   3. Removing redundant array creation from the logging method towards the bottom.
   4. To throw more specific exceptions instead of Runtime exceptions.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {

Review comment:
       Here's the null check is not necessary either.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       This could be replaced with lambda as well to be more compact.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {

Review comment:
       Could you please extract this while loop into a separate method?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Could you please remove _false_ from here?

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Yes, I was talking about initializing the boolean to false is redundant, it adds noise to the code. I think we can rely on that Java developers know how booleans are initialized.
   If we followed this principle we would make every interface method public abstract.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Yes, I was talking about initializing the boolean to false is redundant, it adds noise to the code. I think we can rely on that Java developers know how booleans are initialized.
   If we followed this principle we would make every interface method public abstract for instance.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       The value is to differentiate _relationships_ from _RELATIONSHIPS_. From their name you cannot see the difference.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       3. Since _void log(String msg, Object... os)_ methods are added, _void trace(String msg, Object[] os)_ should be deprecated. The object array wrapping has become redundant and shall be removed with no remorse and forever, whenever seen.
   

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       My IDEA actually is showing it to me: "Redundant array creation for calling varargs method ". Yours isn't?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663999363



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       _instanceof_ returns _false_ for _nulls_, I think you can remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664276040



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Yes, good idea. (But for to make it easier to follow, I will use the `Set::stream` instead of `Collection::stream`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663993537



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;

Review comment:
       Would you please reorder the modifiers to comply with JLS?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r676856523



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+
+                        final Map<String, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<String, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       Is there a reason that we are enforcing this? I don't think it we should. It makes perfect sense to me to return an integer value or a boolean value to denote the partition - or a `null` value. I don't think this check / throwing of an Exception really buys us anything, but it forces users to write potentially more complex code. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664287047



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Could you please elaborate? If you are pointing towards the explicit assignment, that is intentional for easier understanding




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r676859358



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+
+                        final Map<String, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<String, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!outgoingFlowFiles.containsKey(partition)) {

Review comment:
       Can make this more efficient and arguably cleaner - instead of check if it contains key and if not, populating it, followed by calling HashMap.get(), could use the approach:
   ```
   RecordSetWriter writer = recordSetWriters.get(partition);
   if (writer == null) {
      writer = ...
      // populate maps
   }
   writer.write(record);
   ```
   Performing the extra HashMap.get() may seem negligible in terms of performance, but HashMap.get can be pretty expensive, as it has to calculate its own version of a hash code for every get(). So even though String.hashCode() just returns a value, HashMap.get() will re-hash that value, which is fairly expensive. And when you consider a single FlowFile may well have millions of Records, that can amount to adding a few seconds to the processing of a FlowFile.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+
+                        final Map<String, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<String, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!outgoingFlowFiles.containsKey(partition)) {
+                                    final FlowFile outgoingFlowFile = session.create(incomingFlowFile);
+                                    final OutputStream out = session.write(outgoingFlowFile);
+                                    final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+
+                                    writer.beginRecordSet();
+                                    outgoingFlowFiles.put(partition, outgoingFlowFile);
+                                    recordSetWriters.put(partition, writer);
+                                }
+
+                                recordSetWriters.get(partition).write(record);
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned be an instance of String");
+                            }
+                        }
+
+                        // Sending outgoing flow files
+                        int fragmentIndex = 0;
+
+                        for (final String partition : outgoingFlowFiles.keySet()) {
+                            final RecordSetWriter writer = recordSetWriters.get(partition);
+                            final FlowFile outgoingFlowFile = outgoingFlowFiles.get(partition);
+
+                            final Map<String, String> attributes = new HashMap<>(incomingFlowFile.getAttributes());
+                            attributes.put("mime.type", writer.getMimeType());
+                            attributes.put("partition", partition);
+                            attributes.put("fragment.index", String.valueOf(fragmentIndex));
+                            attributes.put("fragment.count", String.valueOf(outgoingFlowFiles.size()));
+
+                            try {
+                                final WriteResult finalResult = writer.finishRecordSet();
+                                final int outgoingFlowFileRecords = finalResult.getRecordCount();
+                                attributes.put("record.count", String.valueOf(outgoingFlowFileRecords));
+                                writer.close();
+                            } catch (final IOException e) {
+                                throw new ProcessException("Resources used for record writing might not be closed", e);
+                            }
+
+                            session.putAllAttributes(outgoingFlowFile, attributes);
+                            session.transfer(outgoingFlowFile, RELATIONSHIP_SUCCESS);
+                            fragmentIndex++;
+                        }
+
+                        session.adjustCounter("Record Processed", index, false);
+                    } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) {
+                        throw new ProcessException("Failed to parse incoming FlowFile", e);
+                    }
+                }
+            });
+
+            return true;
+        } catch (final Exception e) {
+            getLogger().error("Error during routing records", e);

Review comment:
       `getLogger().error("Failed to route records for {}", flowFile, e);` perhaps? Is good to include the FlowFile in the arguments to the logger. There's a Jira already to add the ability to capture the FlowFile from the arguments in order to do something with them for bulletins, etc. Plus it just makes it easier to track through the logs.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {

Review comment:
       @simonbence my concern there is that many users don't read documentation very carefully. Especially when it gets really verbose. I think in general, when considering usability there's a bit of a scale that goes from "Easy to use" to "More Powerful." It's not always the case - you can have things that are both super powerful and super easy to use. But for quite a while, when such a dynamic exists, NiFi has leaned too much toward "More Powerful" I think. I think we need to lean more toward "Easy to use." This is the reason that I would lean toward not including the processor. I would avoid adding things that are likely to introduce confusion or are generally not consistent with the way the rest of the processors work - unless there's a really good reason to "buck the trend." Here, I think there is a reason. But I don't think it's a "really good" reason.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedRecordProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<Relationship, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!outgoingFlowFiles.containsKey(outgoingRelationship.get())) {

Review comment:
       Same comment as above, checking `containsKey` vs. just `get` and check for `null`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664384302



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Yes, I was talking about initializing the boolean to false is redundant, it adds noise to the code. I think we can rely on that Java developers know how booleans are initialized.
   If we followed this principle we would make every interface method public abstract for instance.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#issuecomment-925806967


   Thanks @simonbence! I made a couple of fixes to typos in the docs & a couple of minor clarifications. And addressed a minor issue with handling arrays as partitions in the ScriptedPartitionRecord, but otherwise all looks good to me! Thanks for all of your work on these processors. I think they will be extremely powerful and easy to use. Looking forward to using them myself! +1 merged to main.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664290717



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(

Review comment:
       As the bulk of the method is an anonymous implementation. Splitting the anonymous implementation into parts,  as you suggest might work, but from my perspective, it would have more drawback than merit. If I extend it into the anonymous class as private methods, we do not win much, and if I move the util methods to the enclosing class as static ones, the logic will be pretty fragmented. A lot happens here, which would be good to not complicate further with splitting (in this case, not in general). Also please note that, the implementations I added are slightly based on the already existing `ScriptedTransformRecord`, which I tried to follow in form and function as far as it is possible




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664394408



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       3. Since _void log(String msg, Object... os)_ methods are added, _void trace(String msg, Object[] os)_ should be deprecated. The object array wrapping has become redundant and shall be removed with no remorse and forever, whenever seen.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r667001952



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);

Review comment:
       You are perfectly right, I missed that this is an incrementing method and not a setting one.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r648689913



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to partition and route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines which partition the given record should end up. " +
+        "Partitions are defined as dynamic properties: dynamic property names are serving as partitions known by the processor. " +

Review comment:
       I think it's important here to avoid conflating the ideas of 'partitioning' and 'routing'. I think this should operate in much the same way as PartitionRecord - 3 relationships: success, failure, original. An attribute would be added indicating the partition and all FlowFiles would be routed to success. If routing is desirable, it can be done via RouteOnAttribute. I would avoid any dynamic properties at all.
   
   This results in a much simpler processor (code-wise) and a cleaner, simpler, more consistent user experience.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.ListRecordSet;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptEngine scriptEngine = pollScriptEngine();
+        if (scriptEngine == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailedRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptEngine(scriptEngine);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailedRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, List<Record>> outgoingRecords = new HashMap<>();
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!outgoingRecords.containsKey(outgoingRelationship.get())) {
+                                        outgoingRecords.put(outgoingRelationship.get(), new LinkedList<>());
+                                    }
+
+                                    outgoingRecords.get(outgoingRelationship.get()).add(record);
+                                } else {
+                                    getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue));
+                                }
+                            } else {
+                                throw new ProcessException("Script result is not applicable: " + String.valueOf(evaluatedValue));

Review comment:
       It's probably not going to be very clear to the user what this statement means: "is not applicable." Perhaps it makes more sense to use a message such as: "Script returned a value of " + evaluatedValue + " but this Processor requires that the object returned by an instance of " + getScriptReturnType()

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedPartitionRecord.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestScriptedPartitionRecord extends TestScriptedRouterProcessor {
+    private static final String SCRIPT =

Review comment:
       Is best to externalize this into a file in `src/test/resources`

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestScriptedRouterProcessor.java
##########
@@ -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.
+ */
+package org.apache.nifi.processors.script;
+
+import org.apache.nifi.processor.Processor;
+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.MockRecordWriter;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Before;
+
+import java.util.UUID;
+
+abstract class TestScriptedRouterProcessor {
+    private static final String HEADER = "header§";
+
+    protected TestRunner testRunner;
+    protected MockRecordParser recordReader;
+    protected MockRecordWriter recordWriter;
+    protected String incomingFlowFileContent;
+
+    @Before
+    public void setUp() throws Exception {
+        testRunner = TestRunners.newTestRunner(givenProcessorType());
+
+        testRunner.setProperty(ScriptedTransformRecord.RECORD_READER, "record-reader");
+        testRunner.setProperty(ScriptedTransformRecord.RECORD_WRITER, "record-writer");
+
+        recordReader = new MockRecordParser();
+        recordReader.addSchemaField("first", RecordFieldType.INT);
+        recordReader.addSchemaField("second", RecordFieldType.STRING);
+
+        recordWriter = new MockRecordWriter(HEADER);
+
+        testRunner.addControllerService("record-reader", recordReader);
+        testRunner.addControllerService("record-writer", recordWriter);
+
+        testRunner.setProperty(ScriptingComponentUtils.SCRIPT_BODY, getScript());
+        testRunner.setProperty(ScriptedTransformRecord.LANGUAGE, "Groovy");
+
+        testRunner.enableControllerService(recordReader);
+        testRunner.enableControllerService(recordWriter);
+
+        incomingFlowFileContent = UUID.randomUUID().toString();
+    }
+
+    protected void whenTriggerProcessor() {
+        testRunner.enqueue(incomingFlowFileContent);
+        testRunner.run();
+    }
+
+    protected void thenIncomingFlowFileIsRoutedToOriginal() {
+        testRunner.assertTransferCount(getOriginalRelationship(), 1);
+        testRunner.assertTransferCount(getFailedRelationship(), 0);
+        Assert.assertEquals(incomingFlowFileContent, testRunner.getFlowFilesForRelationship(getOriginalRelationship()).get(0).getContent());
+    }
+
+    protected void thenIncomingFlowFileIsRoutedToFailed() {
+        testRunner.assertTransferCount(getOriginalRelationship(), 0);
+        testRunner.assertTransferCount(getFailedRelationship(), 1);
+        Assert.assertEquals(incomingFlowFileContent, testRunner.getFlowFilesForRelationship(getFailedRelationship()).get(0).getContent());
+    }
+
+    /**
+     * Generates the expected flow file content based on the records. Results the same format as the {@code MockRecordWriter} uses.
+     */
+    protected String givenExpectedFlowFile(final Object[][] records) {

Review comment:
       Would consider using Object[]... as the arg type. Makes for cleaner usage.
   Would also name this method something like `buildFlowFileContent` - `givenExpectedFlowFile` says nothing about what the method does.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedValidateRecord.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "validate", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+        "This processor provides the ability to validate records in FlowFiles using the user-provided script. " +
+        "The script is expected to have a record as incoming argument and return with a boolean value. " +
+        "Based on this result, the processor categorizes the records as \"valid\" or \"invalid\" and routes them to the respected relationship in batch. " +

Review comment:
       I think you meant 'respective' relationship instead of 'respected'

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.ListRecordSet;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptEngine scriptEngine = pollScriptEngine();
+        if (scriptEngine == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailedRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptEngine(scriptEngine);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailedRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, List<Record>> outgoingRecords = new HashMap<>();

Review comment:
       One of the main benefits of record-oriented processors is that they allow users to send in huge flowfiles containing thousands or millions of Records. We should never have a processor that buffers Records (with the exception of buffering up some limited number in order to push a batch to an external system, for example).
   
   Rather than keeping a `Map<Relationship, List<Record>>` we should be holding a `Map<Relationship, RecordSetWriter>`. Then, for each Record, it should be written to the appropriate RecordSetWriter and not held in memory.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedValidateRecord.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "validate", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+        "This processor provides the ability to validate records in FlowFiles using the user-provided script. " +
+        "The script is expected to have a record as incoming argument and return with a boolean value. " +
+        "Based on this result, the processor categorizes the records as \"valid\" or \"invalid\" and routes them to the respected relationship in batch. " +
+        "Additionally the original FlowFile will be routed to the \"original\" relationship or in case of unsuccessful processing, to the \"failed\" relationship."
+)
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord",
+        "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedValidateRecord extends ScriptedRouterProcessor<Boolean> {
+
+    static final Relationship RELATIONSHIP_VALID = new Relationship.Builder()
+            .name("valid")
+            .description(
+                "FlowFile containing the valid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no valid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_INVALID = new Relationship.Builder()
+            .name("invalid")
+            .description(
+                "FlowFile containing the invalid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no invalid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the FlowFiles might routed to \"valid\" and \"invalid\" relationships.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILED = new Relationship.Builder()
+            .name("failed")

Review comment:
       The typical naming convention is to use 'failure' rather than 'failed'

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.ListRecordSet;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptEngine scriptEngine = pollScriptEngine();
+        if (scriptEngine == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailedRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptEngine(scriptEngine);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailedRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, List<Record>> outgoingRecords = new HashMap<>();
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);

Review comment:
       Probably worth logging at a debug level here something to the effect of:
   `getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664390340



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       The value is to differentiate _relationships_ from _RELATIONSHIPS_. From their name you cannot see the difference.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664299724



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       This processor provides the possibility to dynamically add routes, contrary to most of the processors. As processor instances might be called from multiple threads, we need to ensure thread safety. For your second question: I do not see the value of the renaming.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r666523972



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedFilterRecord.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "filter", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+    "This processor provides the ability to filter records out from FlowFiles using the user-provided script. " +
+    "Every record will be evaluated by the script which must return with a boolean value. " +
+    "Records with \"true\" result will be routed to the \"matching\" relationship in a batch. " +
+    "Other records will be filtered out."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedRouteRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedFilterRecord extends ScriptedRouterProcessor<Boolean> {
+    static final Relationship RELATIONSHIP_MATCHING = new Relationship.Builder()
+            .name("matching")
+            .description(
+                "Matching records of the original FlowFile will be routed to this relationship. " +
+                "If there are no matching records, no FlowFile will be routed here."
+            )
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the number of filtered or remaining records.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILED = new Relationship.Builder()
+            .name("failed")

Review comment:
       Should use 'failure' instead of 'failed' to adhere to the standard naming convensions.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();

Review comment:
       Again, I think the abstraction of this RecordBatchingProcessorFlowFileBuilder should be eliminated. All that's really needed is the RecordSetWriter.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);

Review comment:
       I think this logic is incorrect. If you have 3 records, then for the first record you'll adjust the counter by 1. The second time you'll adjust the counter by 2, and the third time you'll adjust it by 3. So you'll have called adjustCounter for 1 + 2 + 3 = 6 instead of adjustCounter(3).
   Should either use `session.adjustCounter("Records Processed", 1, false);` or wait until the end and call `session.adjustCounter("Records Processed", writeResult.getRecordCount(), false);` I would recommend the latter, as it keeps the logic cleaner - do the processing, and at the end handle updating the counters.
   
   Would also make sure that the name is Records with an s.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);
+
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of String");
+                            }
+                        }
+
+                        // Sending outgoing flow files
+                        int fragmentIndex = 1;
+
+                        for (final Map.Entry<String, RecordBatchingProcessorFlowFileBuilder> entry : recordSetFlowFileBuilders.entrySet()) {

Review comment:
       Based on how this is used, I would eliminate this RecordSetFlowFileBuilder class entirely - you really just need a `Map<String, RecordSetWriter>`. And perhaps a parallel `Map<String, FlowFile>` that contains the output FlowFiles.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {

Review comment:
       I think the other processors are very valuable. But I question the necessity of this Processor. I find the configuration very confusing and not user friendly. The user has to understand the possible values that the script could return, and then the user has to go in and redefine them as property keys and map each of those possible values to a relationship. It feels awkward and creates a tight coupling between a 'partition' and a Relationship.
   
   The more conventional approach would be to just use the partition processor. Then, if they want to route based on that, use RouteOnAttribute. The benefits there are that RouteOnAttribute is far more powerful as it enables the full power of Expression Language, it's already well understood, it provides the ability to inspect the attribute value that will be used for routing, which makes it simpler to use in a dataflow, and it decouples the concepts of routes and partitions.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);
+
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of String");
+                            }
+                        }
+
+                        // Sending outgoing flow files
+                        int fragmentIndex = 1;
+
+                        for (final Map.Entry<String, RecordBatchingProcessorFlowFileBuilder> entry : recordSetFlowFileBuilders.entrySet()) {
+                            final String partitionName = entry.getKey();
+                            final RecordBatchingProcessorFlowFileBuilder builder = entry.getValue();
+
+                            FlowFile outgoingFlowFile = builder.build();
+                            outgoingFlowFile = session.putAttribute(outgoingFlowFile, "partition", partitionName);

Review comment:
       Calls to `session.putAttribute` are expensive. They create a full copy of the HashMap that holds attributes. This method should not be used unless only adding a single attribute. If adding multiple attributes, should instead create a Map<String, String>, add all necessary attributes, and then use `session.putAllAttributes`

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!recordSetFlowFileBuilders.containsKey(outgoingRelationship.get())) {
+                                        recordSetFlowFileBuilders.put(outgoingRelationship.get(), new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                    }
+
+                                    final int recordCount = recordSetFlowFileBuilders.get(outgoingRelationship.get()).addRecord(record);
+                                    session.adjustCounter("Record Processed", recordCount, false);
+                                } else {
+                                    getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue));
+                                }
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of " + scriptResultType.getSimpleName());

Review comment:
       'returned be an instance' rather than 'returned by an instance'

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedFilterRecord.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "filter", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+    "This processor provides the ability to filter records out from FlowFiles using the user-provided script. " +
+    "Every record will be evaluated by the script which must return with a boolean value. " +
+    "Records with \"true\" result will be routed to the \"matching\" relationship in a batch. " +
+    "Other records will be filtered out."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedRouteRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedFilterRecord extends ScriptedRouterProcessor<Boolean> {
+    static final Relationship RELATIONSHIP_MATCHING = new Relationship.Builder()
+            .name("matching")

Review comment:
       Should use 'matched' or 'success' instead of 'matching' in order to stick to standard naming conventions. Would recommend 'success'.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);
+
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of String");

Review comment:
       I think that's supposed to be 'be an instance' instead of 'by an instance'

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!recordSetFlowFileBuilders.containsKey(outgoingRelationship.get())) {
+                                        recordSetFlowFileBuilders.put(outgoingRelationship.get(), new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                    }
+
+                                    final int recordCount = recordSetFlowFileBuilders.get(outgoingRelationship.get()).addRecord(record);
+                                    session.adjustCounter("Record Processed", recordCount, false);

Review comment:
       Same comment as the other processor. Should call it 'Records' with an s - but the same logic error appears to be present here - the counter is being incremented n + (n-1) + (n-2) + ... times instead of n times.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedProcessor.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.script;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.script.ScriptingComponentHelper;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+abstract class ScriptedProcessor extends AbstractProcessor implements Searchable {

Review comment:
       Recommend calling this `AbstractScriptedRecordProcessor` or something along those lines, in order to convey the intent fo this class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r663994801



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Minor: You can use method reference here:
   
   ```suggestion
           final Set<String> attributeNames = attributes.stream().map(Map::keySet).flatMap(Collection::stream).collect(Collectors.toSet());
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664007567



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(

Review comment:
       This method is quite complex. What do you think of extracting the parts at the lines 184-204 and 209-218 into separate methods to increase readability?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r692029232



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+
+                        final Map<String, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<String, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       I was (maybe wrongly) assuming that that we expect a string value as partition. But you are right, any value might be good.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664076464



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {

Review comment:
       Could you please extract this while loop into a separate method?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 closed pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 closed pull request #4948:
URL: https://github.com/apache/nifi/pull/4948


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#issuecomment-901893094


   I have updated the template https://gist.github.com/simonbence/d0ce94d79025213aeec61bc45604ca17


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664068504



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedValidateRecord.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+@Tags({"record", "validate", "script", "groovy", "jython", "python"})
+@CapabilityDescription(
+        "This processor provides the ability to validate records in FlowFiles using the user-provided script. " +
+        "The script is expected to have a record as incoming argument and return with a boolean value. " +
+        "Based on this result, the processor categorizes the records as \"valid\" or \"invalid\" and routes them to the respective relationship in batch. " +
+        "Additionally the original FlowFile will be routed to the \"original\" relationship or in case of unsuccessful processing, to the \"failed\" relationship."
+)
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedPartitionRecord"
+})
+public class ScriptedValidateRecord extends ScriptedRouterProcessor<Boolean> {
+
+    static final Relationship RELATIONSHIP_VALID = new Relationship.Builder()
+            .name("valid")
+            .description(
+                "FlowFile containing the valid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no valid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_INVALID = new Relationship.Builder()
+            .name("invalid")
+            .description(
+                "FlowFile containing the invalid records from the incoming FlowFile will be routed to this relationship. " +
+                "If there are no invalid records, no FlowFile will be routed to this Relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the FlowFiles might routed to \"valid\" and \"invalid\" relationships.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("In case of any issue during processing the incoming flow file, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();

Review comment:
       Just for consistency: _RELATIONSHIPS_ may be final.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664004574



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Would you please remove the redundant initializer here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664013449



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedProcessor.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.script;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.script.ScriptingComponentHelper;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+abstract class ScriptedProcessor extends AbstractProcessor implements Searchable {
+    protected static final String PYTHON_SCRIPT_LANGUAGE = "python";
+    protected static final Set<String> SCRIPT_OPTIONS = ScriptingComponentUtils.getAvailableEngines();
+
+    protected volatile String scriptToRun = null;
+    protected final AtomicReference<CompiledScript> compiledScriptRef = new AtomicReference<>();
+    private final ScriptingComponentHelper scriptingComponentHelper = new ScriptingComponentHelper();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+
+    @OnScheduled
+    public void setup(final ProcessContext context) throws IOException {
+        if (!scriptingComponentHelper.isInitialized.get()) {
+            scriptingComponentHelper.createResources(false);
+        }
+
+        scriptingComponentHelper.setupVariables(context);
+        scriptToRun = scriptingComponentHelper.getScriptBody();
+
+        if (scriptToRun == null && scriptingComponentHelper.getScriptPath() != null) {
+            try (final FileInputStream scriptStream = new FileInputStream(scriptingComponentHelper.getScriptPath())) {
+                scriptToRun = IOUtils.toString(scriptStream, Charset.defaultCharset());
+            }
+        }
+
+        // Create a script runner for each possible task
+        final int maxTasks = context.getMaxConcurrentTasks();
+        scriptingComponentHelper.setupScriptRunners(maxTasks, scriptToRun, getLogger());
+
+        // Always compile when first run
+        compiledScriptRef.set(null);
+    }
+
+    protected ScriptEvaluator createEvaluator(final ScriptEngine scriptEngine, final FlowFile flowFile) throws ScriptException {
+        if (PYTHON_SCRIPT_LANGUAGE.equalsIgnoreCase(scriptEngine.getFactory().getLanguageName())) {
+            final CompiledScript compiledScript = getOrCompileScript((Compilable) scriptEngine, scriptToRun);
+            return new PythonScriptEvaluator(scriptEngine, compiledScript, flowFile, getLogger());
+        }
+
+        return new InterpretedScriptEvaluator(scriptEngine, scriptToRun, flowFile, getLogger());
+    }
+
+    private CompiledScript getOrCompileScript(final Compilable scriptEngine, final String scriptToRun) throws ScriptException {
+        final CompiledScript existing = compiledScriptRef.get();
+        if (existing != null) {
+            return existing;
+        }
+
+        final CompiledScript compiled = scriptEngine.compile(scriptToRun);
+        final boolean updated = compiledScriptRef.compareAndSet(null, compiled);
+        if (updated) {
+            return compiled;
+        }
+
+        return compiledScriptRef.get();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        return scriptingComponentHelper.customValidate(validationContext);
+    }
+
+    @Override
+    public Collection<SearchResult> search(final SearchContext context) {
+        return ScriptingComponentUtils.search(context, getLogger());
+    }
+
+    protected static Bindings setupBindings(final ScriptEngine scriptEngine) {
+        Bindings bindings = scriptEngine.getBindings(ScriptContext.ENGINE_SCOPE);
+        if (bindings == null) {
+            bindings = new SimpleBindings();
+        }
+
+        scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
+
+        return bindings;
+    }
+
+    protected ScriptRunner pollScriptRunner() {
+        return scriptingComponentHelper.scriptRunnerQ.poll();
+    }
+
+    protected void offerScriptRunner(ScriptRunner scriptRunner) {
+        scriptingComponentHelper.scriptRunnerQ.offer(scriptRunner);

Review comment:
       Is it possible at any point that we cannot add the _scriptRunner_ to the queue? If so, should the _false_ return value of _offer_ get handled?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r692004847



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+
+                        final Map<String, FlowFile> outgoingFlowFiles = new HashMap<>();
+                        final Map<String, RecordSetWriter> recordSetWriters = new HashMap<>();
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!outgoingFlowFiles.containsKey(partition)) {

Review comment:
       I was not considered the multitude of the records, thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664303269



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {

Review comment:
       That is correct, and just like above I prefer to show the intent explicitly




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r666550072



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {
+                                final Optional<Relationship> outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue));
+
+                                if (outgoingRelationship.isPresent()) {
+                                    if (!recordSetFlowFileBuilders.containsKey(outgoingRelationship.get())) {
+                                        recordSetFlowFileBuilders.put(outgoingRelationship.get(), new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                    }
+
+                                    final int recordCount = recordSetFlowFileBuilders.get(outgoingRelationship.get()).addRecord(record);
+                                    session.adjustCounter("Record Processed", recordCount, false);
+                                } else {
+                                    getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue));
+                                }
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of " + scriptResultType.getSimpleName());
+                            }
+                        }
+
+                        // Sending outgoing flow files
+                        recordSetFlowFileBuilders.forEach((relationship, builder) -> session.transfer(builder.build(), relationship));
+                    } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) {
+                        throw new ProcessException("Failed to parse incoming FlowFile", e);
+                    }
+                }
+            });
+
+            return true;
+        } catch (final Exception e) {
+            getLogger().error("Error during routing records", e);
+            return false;
+        }
+    }
+
+    /**
+     * @return Returns with the relationship used for route the incoming FlowFile in case of successful processing.
+     */
+    protected abstract Relationship getOriginalRelationship();
+
+    /**
+     * @return Returns with the relationship used for route the incoming FlowFile in case of unsuccessful processing.
+     */
+    protected abstract Relationship getFailureRelationship();
+
+    /**
+     * Returns a relationship based on the script's result value. As the script uses a given record as input, this helps
+     * to dissolve the result value for the routing.
+     *
+     * @param scriptResult The value returned by the script.
+     *
+     * @return Returns with a relationship if there is one to determine based on the value. If it is not possible to determine
+     * an {code Optional#empty} is expected. Records with empty relationship will not be routed into any relationship (expect for

Review comment:
       Think that was supposed to be 'except for' instead of 'expect for'




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r669899338



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {
+                                final String partition = (String) evaluatedValue;
+
+                                if (!recordSetFlowFileBuilders.containsKey(partition)) {
+                                    recordSetFlowFileBuilders.put(partition, new RecordBatchingProcessorFlowFileBuilder(incomingFlowFile, session, recordSetWriterFactory));
+                                }
+
+                                final int recordCount = recordSetFlowFileBuilders.get(partition).addRecord(record);
+                                session.adjustCounter("Record Processed", recordCount, false);
+
+                            } else {
+                                throw new ProcessException("Script returned a value of " + evaluatedValue
+                                        + " but this Processor requires that the object returned by an instance of String");
+                            }
+                        }
+
+                        // Sending outgoing flow files
+                        int fragmentIndex = 1;
+
+                        for (final Map.Entry<String, RecordBatchingProcessorFlowFileBuilder> entry : recordSetFlowFileBuilders.entrySet()) {

Review comment:
       As based on your suggestions I removed the "attribute merging functions", these will be enough, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664002776



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       What do you think of moving this up as a separate method e.g. _createRecordSetWriterFactory_ to _ScriptedProcessor_? This is duplicated in _ScriptedRouterProcessor_ 142-148




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664015010



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
+    private final Map<String, Relationship> routes = new ConcurrentHashMap<>();
+
+    public ScriptedRouteRecord() {
+        super(String.class);
+    }
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        relationships.set(new HashSet<>(RELATIONSHIPS));
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships.get();
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .required(false)
+            .name(propertyDescriptorName)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .dynamic(true)
+            .build();
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (descriptor.isDynamic()) {
+            final Set<Relationship> relationships = new HashSet<>(this.relationships.get());
+            final String routeName = descriptor.getName();
+
+            if (shouldDeleteDynamicRelationship(routeName, oldValue)) {
+                relationships.remove(new Relationship.Builder().name(oldValue).build());
+            }
+
+            if (newValue == null) {
+                routes.remove(routeName);
+            } else {
+                final Relationship newRelationship = new Relationship.Builder().name(newValue).build();
+                routes.put(routeName, newRelationship);
+                relationships.add(newRelationship);
+            }
+
+            this.relationships.set(relationships);
+        }
+    }
+
+    private boolean shouldDeleteDynamicRelationship(final String routeName, final String oldValue) {
+        // If no further route points to the same relationship and it is not a static relationship, it must be removed
+        final Set<String> staticRelationships = RELATIONSHIPS.stream().map(r -> r.getName()).collect(Collectors.toSet());

Review comment:
       Minor: Lambda can be replaced with _Relationship::getName_.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664384302



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Yes, I was talking about initializing the boolean to false is redundant, it adds noise to the code. I think we can rely on that Java developers know how booleans are initialized.
   If we followed this principle we would make every interface method public abstract.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664295102



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedProcessor.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.script;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.script.ScriptingComponentHelper;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+abstract class ScriptedProcessor extends AbstractProcessor implements Searchable {
+    protected static final String PYTHON_SCRIPT_LANGUAGE = "python";
+    protected static final Set<String> SCRIPT_OPTIONS = ScriptingComponentUtils.getAvailableEngines();
+
+    protected volatile String scriptToRun = null;
+    protected final AtomicReference<CompiledScript> compiledScriptRef = new AtomicReference<>();
+    private final ScriptingComponentHelper scriptingComponentHelper = new ScriptingComponentHelper();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+
+    @OnScheduled
+    public void setup(final ProcessContext context) throws IOException {
+        if (!scriptingComponentHelper.isInitialized.get()) {
+            scriptingComponentHelper.createResources(false);
+        }
+
+        scriptingComponentHelper.setupVariables(context);
+        scriptToRun = scriptingComponentHelper.getScriptBody();
+
+        if (scriptToRun == null && scriptingComponentHelper.getScriptPath() != null) {
+            try (final FileInputStream scriptStream = new FileInputStream(scriptingComponentHelper.getScriptPath())) {
+                scriptToRun = IOUtils.toString(scriptStream, Charset.defaultCharset());
+            }
+        }
+
+        // Create a script runner for each possible task
+        final int maxTasks = context.getMaxConcurrentTasks();
+        scriptingComponentHelper.setupScriptRunners(maxTasks, scriptToRun, getLogger());
+
+        // Always compile when first run
+        compiledScriptRef.set(null);
+    }
+
+    protected ScriptEvaluator createEvaluator(final ScriptEngine scriptEngine, final FlowFile flowFile) throws ScriptException {
+        if (PYTHON_SCRIPT_LANGUAGE.equalsIgnoreCase(scriptEngine.getFactory().getLanguageName())) {
+            final CompiledScript compiledScript = getOrCompileScript((Compilable) scriptEngine, scriptToRun);
+            return new PythonScriptEvaluator(scriptEngine, compiledScript, flowFile, getLogger());
+        }
+
+        return new InterpretedScriptEvaluator(scriptEngine, scriptToRun, flowFile, getLogger());
+    }
+
+    private CompiledScript getOrCompileScript(final Compilable scriptEngine, final String scriptToRun) throws ScriptException {
+        final CompiledScript existing = compiledScriptRef.get();
+        if (existing != null) {
+            return existing;
+        }
+
+        final CompiledScript compiled = scriptEngine.compile(scriptToRun);
+        final boolean updated = compiledScriptRef.compareAndSet(null, compiled);
+        if (updated) {
+            return compiled;
+        }
+
+        return compiledScriptRef.get();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        return scriptingComponentHelper.customValidate(validationContext);
+    }
+
+    @Override
+    public Collection<SearchResult> search(final SearchContext context) {
+        return ScriptingComponentUtils.search(context, getLogger());
+    }
+
+    protected static Bindings setupBindings(final ScriptEngine scriptEngine) {
+        Bindings bindings = scriptEngine.getBindings(ScriptContext.ENGINE_SCOPE);
+        if (bindings == null) {
+            bindings = new SimpleBindings();
+        }
+
+        scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
+
+        return bindings;
+    }
+
+    protected ScriptRunner pollScriptRunner() {
+        return scriptingComponentHelper.scriptRunnerQ.poll();
+    }
+
+    protected void offerScriptRunner(ScriptRunner scriptRunner) {
+        scriptingComponentHelper.scriptRunnerQ.offer(scriptRunner);

Review comment:
       In theory, this is a possibility. If the `BlockingQueue` would be full, we would got a `null`, even worse, we would loose the instance, as we would not held reference on that anymore. Luckily, the script runner logic is fabricated on a manner that it works with a predefined set of runners, which are `pull`-ed and after usage, `offer`-ed back to the queue. The situation you are mentioning would assume that somewhere a new instance is being created which is not the case.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664285857



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       As the implementation is quite big, this helps reading and keeping boundaries




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r661422099



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to partition and route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines which partition the given record should end up. " +
+        "Partitions are defined as dynamic properties: dynamic property names are serving as partitions known by the processor. " +

Review comment:
       You are right, I checked back the existing partition processor. However I see the merit of both behaviour so I decided to separate the two different business logic into two different processors. One is for partitioning, which mimics the behaviour of the existing `PartitionRecord` from most of the aspects (but of course depending on the script evaluation) and I also added a `ScriptedRoutingRecord` which based on the script result and the pre-configured variables will transfer the flow files into different routes without additional steps.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r666520891



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    private final FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();

Review comment:
       This is a complex and potentially very expensive data structure. It's holding a lot of attributes that we then going to ignore.
   I don't think we actually need to keep track of any of these attributes at all, to be honest. When the final Record has been written to the Record Writer, we should be calling
   ```
   WriteResult result = writer.finishRecordSet();
   ```
   It looks like this happens here in the `build()` method. We do not need to keep any of the intermediate values. We also shouldn't need to keep count of the number of records written, as that's also returned in the WriteResult.
   
   I'll need to continue the review to understand exactly how this is being used, but I'm not sure that this class is even needed at all. Typically, we will just create a RecordSetWriter, write the records, and then call finishRecordSet(). That call to finishRecordSet() should give us all that we need that is being tracked here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664395395



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       My IDEA actually is showing it to me: "Redundant array creation for calling varargs method ". Yours isn't?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664302911



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       - As for points 1 and 2, the same applies as above.
   - For point 3> I am not sure which ones you are referring, but the ones in lines 254, 262 and 270: the content of the array is different in every case thus, we cannot spare them
   - `RuntimeException`: in general I agree with you and it could be a good later improvement point but as this ticket ads a bulk of new functionality I would find it unlucky to functionally change already existing behaviour. That easily ends up with regression issues.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r691867927



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedRecordProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static final Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {

Review comment:
       This was a copy-paste from the `ScriptedTransformRecord` which was my template in many senses. But you are perfectly right, especially that now it is used in multiple places. I made the change and now it throws a `ProcessException`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664276040



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/RecordBatchingProcessorFlowFileBuilder.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.Record;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class contains all the information necessary to prepare an outgoing flow file.
+ */
+final class RecordBatchingProcessorFlowFileBuilder {
+    private final ProcessSession session;
+    private final FlowFile incomingFlowFile;
+    final private FlowFile outgoingFlowFile;
+    private final OutputStream out;
+    private final RecordSetWriter writer;
+    private final List<Map<String, String>> attributes = new LinkedList<>();
+
+    private int recordCount = 0;
+
+    RecordBatchingProcessorFlowFileBuilder(
+            final FlowFile incomingFlowFile,
+            final ProcessSession session,
+            final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterSupplier
+    ) throws IOException {
+        this.session = session;
+        this.incomingFlowFile = incomingFlowFile;
+        this.outgoingFlowFile = session.create(incomingFlowFile);
+        this.out = session.write(outgoingFlowFile);
+        this.writer = recordSetWriterSupplier.apply(outgoingFlowFile, out);
+        this.writer.beginRecordSet();
+    }
+
+    int addRecord(final Record record) throws IOException {
+        final WriteResult writeResult = writer.write(record);
+        attributes.add(writeResult.getAttributes());
+        recordCount += writeResult.getRecordCount();
+        return recordCount;
+    }
+
+    private Map<String, String> getWriteAttributes() {
+        final Map<String, String> result = new HashMap<>();
+        final Set<String> attributeNames = attributes.stream().map(a -> a.keySet()).flatMap(x -> x.stream()).collect(Collectors.toSet());

Review comment:
       Yes, good idea. (But for to make it easier to follow, I will use the `Set::stream` instead of `Collection::stream`

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && evaluatedValue instanceof String) {

Review comment:
       Checking the null explicitly communicates my intention more clearly

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<String, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };

Review comment:
       I was thinking on this originally but I decided not to do. My key reasons: 1. this behaviour is specific to these two classes and not part of the more generic abstraction `ScriptedProcessor`, thus the abstraction would break. 2. this is not as big amount of code to introduce an intermediate level of abstraction (Note: originally these two classes were not separated)

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(
+            final ProcessContext context,
+            final ProcessSession session,
+            final FlowFile incomingFlowFile,
+            final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       As the implementation is quite big, this helps reading and keeping boundaries

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Could you please elaborate? If you are pointing towards the explicit assignment, that is intentional for easier understanding

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, RELATIONSHIP_FAILURE);
+                return;
+            }
+
+            success = partition(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? RELATIONSHIP_ORIGINAL : RELATIONSHIP_FAILURE);
+    }
+
+    private boolean partition(

Review comment:
       As the bulk of the method is an anonymous implementation. Splitting the anonymous implementation into parts,  as you suggest might work, but from my perspective, it would have more drawback than merit. If I extend it into the anonymous class as private methods, we do not win much, and if I move the util methods to the enclosing class as static ones, the logic will be pretty fragmented. A lot happens here, which would be good to not complicate further with splitting (in this case, not in general). Also please note that, the implementations I added are slightly based on the already existing `ScriptedTransformRecord`, which I tried to follow in form and function as far as it is possible

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedProcessor.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.script;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.script.ScriptingComponentHelper;
+import org.apache.nifi.script.ScriptingComponentUtils;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptContext;
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+abstract class ScriptedProcessor extends AbstractProcessor implements Searchable {
+    protected static final String PYTHON_SCRIPT_LANGUAGE = "python";
+    protected static final Set<String> SCRIPT_OPTIONS = ScriptingComponentUtils.getAvailableEngines();
+
+    protected volatile String scriptToRun = null;
+    protected final AtomicReference<CompiledScript> compiledScriptRef = new AtomicReference<>();
+    private final ScriptingComponentHelper scriptingComponentHelper = new ScriptingComponentHelper();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use parsing the incoming FlowFile into Records")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("Record Writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records after they have been transformed")
+            .required(true)
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .build();
+
+    static final PropertyDescriptor LANGUAGE = new PropertyDescriptor.Builder()
+            .name("Script Engine")
+            .displayName("Script Language")
+            .description("The Language to use for the script")
+            .allowableValues(SCRIPT_OPTIONS)
+            .defaultValue("Groovy")
+            .required(true)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Arrays.asList(
+            RECORD_READER,
+            RECORD_WRITER,
+            LANGUAGE,
+            ScriptingComponentUtils.SCRIPT_BODY,
+            ScriptingComponentUtils.SCRIPT_FILE,
+            ScriptingComponentUtils.MODULES);
+
+
+    @OnScheduled
+    public void setup(final ProcessContext context) throws IOException {
+        if (!scriptingComponentHelper.isInitialized.get()) {
+            scriptingComponentHelper.createResources(false);
+        }
+
+        scriptingComponentHelper.setupVariables(context);
+        scriptToRun = scriptingComponentHelper.getScriptBody();
+
+        if (scriptToRun == null && scriptingComponentHelper.getScriptPath() != null) {
+            try (final FileInputStream scriptStream = new FileInputStream(scriptingComponentHelper.getScriptPath())) {
+                scriptToRun = IOUtils.toString(scriptStream, Charset.defaultCharset());
+            }
+        }
+
+        // Create a script runner for each possible task
+        final int maxTasks = context.getMaxConcurrentTasks();
+        scriptingComponentHelper.setupScriptRunners(maxTasks, scriptToRun, getLogger());
+
+        // Always compile when first run
+        compiledScriptRef.set(null);
+    }
+
+    protected ScriptEvaluator createEvaluator(final ScriptEngine scriptEngine, final FlowFile flowFile) throws ScriptException {
+        if (PYTHON_SCRIPT_LANGUAGE.equalsIgnoreCase(scriptEngine.getFactory().getLanguageName())) {
+            final CompiledScript compiledScript = getOrCompileScript((Compilable) scriptEngine, scriptToRun);
+            return new PythonScriptEvaluator(scriptEngine, compiledScript, flowFile, getLogger());
+        }
+
+        return new InterpretedScriptEvaluator(scriptEngine, scriptToRun, flowFile, getLogger());
+    }
+
+    private CompiledScript getOrCompileScript(final Compilable scriptEngine, final String scriptToRun) throws ScriptException {
+        final CompiledScript existing = compiledScriptRef.get();
+        if (existing != null) {
+            return existing;
+        }
+
+        final CompiledScript compiled = scriptEngine.compile(scriptToRun);
+        final boolean updated = compiledScriptRef.compareAndSet(null, compiled);
+        if (updated) {
+            return compiled;
+        }
+
+        return compiledScriptRef.get();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        return scriptingComponentHelper.customValidate(validationContext);
+    }
+
+    @Override
+    public Collection<SearchResult> search(final SearchContext context) {
+        return ScriptingComponentUtils.search(context, getLogger());
+    }
+
+    protected static Bindings setupBindings(final ScriptEngine scriptEngine) {
+        Bindings bindings = scriptEngine.getBindings(ScriptContext.ENGINE_SCOPE);
+        if (bindings == null) {
+            bindings = new SimpleBindings();
+        }
+
+        scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE);
+
+        return bindings;
+    }
+
+    protected ScriptRunner pollScriptRunner() {
+        return scriptingComponentHelper.scriptRunnerQ.poll();
+    }
+
+    protected void offerScriptRunner(ScriptRunner scriptRunner) {
+        scriptingComponentHelper.scriptRunnerQ.offer(scriptRunner);

Review comment:
       In theory, this is a possibility. If the `BlockingQueue` would be full, we would got a `null`, even worse, we would loose the instance, as we would not held reference on that anymore. Luckily, the script runner logic is fabricated on a manner that it works with a predefined set of runners, which are `pull`-ed and after usage, `offer`-ed back to the queue. The situation you are mentioning would assume that somewhere a new instance is being created which is not the case.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouteRecord.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@Tags({"record", "routing", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription(
+        "This processor provides the ability to route the records of the incoming FlowFile using an user-provided script. " +
+        "The script is expected to handle a record as argument and return with a string value. " +
+        "The returned value defines a route. All routes are bounded to an outgoing relationship where the record will be transferred to. " +
+        "Relationships are defined as dynamic properties: dynamic property names are serving as the name of the route. " +
+        "The value of a dynamic property defines the relationship the given record will be routed into. Multiple routes might point to the same relationship. " +
+        "Creation of these dynamic relationship is managed by the processor. " +
+        "The records, which for the script returned with an unknown relationship name are routed to the \"unmatched\" relationship. " +
+        "The records are batched: for an incoming FlowFile, all the records routed towards a given relationship are batched into one single FlowFile."
+)
+@SeeAlso(classNames = {
+    "org.apache.nifi.processors.script.ScriptedTransformRecord",
+    "org.apache.nifi.processors.script.ScriptedPartitionRecord",
+    "org.apache.nifi.processors.script.ScriptedValidateRecord",
+    "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+@DynamicRelationship(name = "Name from Dynamic Property", description = "FlowFiles that match the Dynamic Property's Attribute Expression Language")
+public class ScriptedRouteRecord extends ScriptedRouterProcessor<String> {
+
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description(
+                "After successful procession, the incoming FlowFile will be transferred to this relationship. " +
+                "This happens regardless the records are matching to a relationship or not.")
+            .build();
+
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failed")
+            .description("In case of any issue during processing the incoming FlowFile, the incoming FlowFile will be routed to this relationship.")
+            .build();
+
+    static final Relationship RELATIONSHIP_UNMATCHED = new Relationship.Builder()
+            .name("unmatched")
+            .description("Records where the script evaluation returns with an unknown partition are routed to this relationship.")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+        RELATIONSHIPS.add(RELATIONSHIP_UNMATCHED);
+    }
+
+    private final AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();

Review comment:
       This processor provides the possibility to dynamically add routes, contrary to most of the processors. As processor instances might be called from multiple threads, we need to ensure thread safety. For your second question: I do not see the value of the renaming.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedTransformRecord.java
##########
@@ -17,7 +17,6 @@
 

Review comment:
       - As for points 1 and 2, the same applies as above.
   - For point 3> I am not sure which ones you are referring, but the ones in lines 254, 262 and 270: the content of the array is different in every case thus, we cannot spare them
   - `RuntimeException`: in general I agree with you and it could be a good later improvement point but as this ticket ads a bulk of new functionality I would find it unlucky to functionally change already existing behaviour. That easily ends up with regression issues.

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {

Review comment:
       That is correct, and just like above I prefer to show the intent explicitly

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       Same as above

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {

Review comment:
       As I argued above, splitting the methods of the anonymous implementations would make this part harder to read and because of this I prefer to leave it as it is. (However, I do understand the cyclomatic complexity is high)

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       Please see above

##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SideEffectFree
+@Tags({"record", "partition", "script", "groovy", "jython", "python", "segment", "split", "group", "organize"})
+@CapabilityDescription("Receives Record-oriented data (i.e., data that can be read by the configured Record Reader) and evaluates the user provided script against "
+        + "each record in the incoming flow file. Each record is then grouped with other records sharing the same partition and a FlowFile is created for each groups of records. " +
+        "Two records shares the same partition if the evaluation of the script results the same return value for both. Those will be considered as part of the same partition.")
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "partition", description = "The partition of the outgoing flow file."),
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer."),
+        @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the partitioned FlowFiles that were created from a single parent FlowFile"),
+        @WritesAttribute(attribute = "fragment.count", description = "The number of partitioned FlowFiles generated from the parent FlowFile")
+})
+@SeeAlso(classNames = {
+        "org.apache.nifi.processors.script.ScriptedTransformRecord",
+        "org.apache.nifi.processors.script.ScriptedRouteRecord",
+        "org.apache.nifi.processors.script.ScriptedValidateRecord",
+        "org.apache.nifi.processors.script.ScriptedFilterRecord"
+})
+public class ScriptedPartitionRecord extends ScriptedProcessor {
+
+    static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles that are successfully partitioned will be routed to this relationship")
+            .build();
+    static final Relationship RELATIONSHIP_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Once all records in an incoming FlowFile have been partitioned, the original FlowFile is routed to this relationship.")
+            .build();
+    static final Relationship RELATIONSHIP_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("If a FlowFile cannot be partitioned from the configured input format to the configured output format, "
+                    + "the unchanged FlowFile will be routed to this relationship")
+            .build();
+
+    private static Set<Relationship> RELATIONSHIPS = new HashSet<>();
+
+    static {
+        RELATIONSHIPS.add(RELATIONSHIP_ORIGINAL);
+        RELATIONSHIPS.add(RELATIONSHIP_SUCCESS);
+        RELATIONSHIPS.add(RELATIONSHIP_FAILURE);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;

Review comment:
       I consider that a different case. Here I see value in verbosity in this level.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664075743



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {

Review comment:
       This could be replaced with lambda as well to be more compact.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#discussion_r664075511



##########
File path: nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.script;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.PushBackRecordSet;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+import javax.script.ScriptEngine;
+import javax.script.ScriptException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+
+@EventDriven
+@SupportsBatching
+@SideEffectFree
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Restricted(restrictions = {
+        @Restriction(requiredPermission = RequiredPermission.EXECUTE_CODE,
+                explanation = "Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
+})
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"),
+        @WritesAttribute(attribute = "record.count", description = "The number of records within the flow file."),
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+public abstract class ScriptedRouterProcessor<T> extends ScriptedProcessor {
+    private final Class<T> scriptResultType;
+
+    /**
+     * @param scriptResultType Defines the expected result type of the user-provided script.
+     */
+    protected ScriptedRouterProcessor(final Class<T> scriptResultType) {
+        this.scriptResultType = scriptResultType;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ScriptRunner scriptRunner = pollScriptRunner();
+        if (scriptRunner == null) {
+            // This shouldn't happen. But just in case.
+            session.rollback();
+            return;
+        }
+
+        boolean success = false;
+
+        try {
+            final ScriptEvaluator evaluator;
+
+            try {
+                final ScriptEngine scriptEngine = scriptRunner.getScriptEngine();
+                evaluator = createEvaluator(scriptEngine, flowFile);
+            } catch (final ScriptException se) {
+                getLogger().error("Failed to initialize script engine", se);
+                session.transfer(flowFile, getFailureRelationship());
+                return;
+            }
+
+            success = route(context, session, flowFile, evaluator);
+        } finally {
+            offerScriptRunner(scriptRunner);
+        }
+
+        session.transfer(flowFile, success ? getOriginalRelationship() : getFailureRelationship());
+    }
+
+    private boolean route(
+        final ProcessContext context,
+        final ProcessSession session,
+        final FlowFile incomingFlowFile,
+        final ScriptEvaluator evaluator
+    ) {
+        final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Map<String, String> originalAttributes = incomingFlowFile.getAttributes();
+
+        try {
+            session.read(incomingFlowFile, new InputStreamCallback() {
+                @Override
+                public void process(final InputStream in) throws IOException {
+                    try (
+                        final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger())
+                    ) {
+                        final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema());
+                        final RecordSet recordSet = reader.createRecordSet();
+                        final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet);
+                        final Map<Relationship, RecordBatchingProcessorFlowFileBuilder> recordSetFlowFileBuilders = new HashMap<>();
+                        final BiFunction<FlowFile, OutputStream, RecordSetWriter> recordSetWriterFactory = (outgoingFlowFile, out) -> {
+                            try {
+                                return writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile);
+                            } catch (final IOException | SchemaNotFoundException e) {
+                                throw new ProcessException("Could not create RecordSetWriter", e);
+                            }
+                        };
+
+                        int index = 0;
+
+                        // Reading in records and evaluate script
+                        while (pushBackSet.isAnotherRecord()) {
+                            final Record record = pushBackSet.next();
+                            final Object evaluatedValue = evaluator.evaluate(record, index++);
+                            getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, index - 1, evaluatedValue);
+
+                            if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) {

Review comment:
       Here's the null check is not necessary either.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #4948: NIFI-8273 Adding Scripted Record processors

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4948:
URL: https://github.com/apache/nifi/pull/4948#issuecomment-907546294


   @simonbence I got a chance to start testing these processors this evening. The first thing that I did was to look at the Usage to understand how to use the processors. Unfortunately, they don't really provide much of any documentation at all. I think each of these needs to have their own additionalDetails.html. This is really important for any & all scripted components because otherwise users are just kind of taking a shot in the dark, guessing how to use it. Would recommend looking at the additionalDetails of ScriptedTransformRecord. Specifically, it will be important to explain how the return values are to be returned. For example, Jython won't allow you to use a `return true` syntax - you have to set `_ = true`, etc. This is important to call out. It's also crucial to call out all of the variable bindings so that users know what they have access to, and to ensure that we provide examples - I would say at least 1 example in Groovy and 1 in Jython would be ideal, as these are by 
 far the most commonly used.
   
   I did also notice that there was an additionalDetails from the ScriptedRouteRecord, which is the one we removed. So we need to make sure that its additionalDetails.html is removed. Additionally, the other Scripted components list ScriptedRouteRecord in their @SeeAlso annotation, so we'll want to remove that also.
   
   In testing, I wanted to see how it handled an Exception being thrown, so I created a simple Groovy script:
   ```
   throw new RuntimeException("Oops")
   ```
   
   The processor (ScriptedValidateRecord in this case) did properly route the incoming FlowFile to failure. However, the error message that I got was not accurate:
   ```
   2021-08-27 21:23:42,129 ERROR [Timer-Driven Process Thread-8] o.a.n.p.script.ScriptedValidateRecord ScriptedValidateRecord[id=8a4da405-017b-1000-54c8-e807155bed3b] Failed to route records for StandardFlowFileRecord[uuid=71d42d80-8e63-4d13-815c-ae0106eeed3b,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1630113179238-1, container=default, section=1], offset=243, length=100],offset=0,name=71d42d80-8e63-4d13-815c-ae0106eeed3b,size=100]: org.apache.nifi.processor.exception.ProcessException: Failed to parse incoming FlowFile
   org.apache.nifi.processor.exception.ProcessException: Failed to parse incoming FlowFile
   	at org.apache.nifi.processors.script.ScriptedRouterProcessor$1.process(ScriptedRouterProcessor.java:194)
   	at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2466)
   	at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2434)
   	at org.apache.nifi.processors.script.ScriptedRouterProcessor.route(ScriptedRouterProcessor.java:126)
   	at org.apache.nifi.processors.script.ScriptedRouterProcessor.onTrigger(ScriptedRouterProcessor.java:107)
   	at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
   	at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1202)
   	at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214)
   	at org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:63)
   	at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: javax.script.ScriptException: javax.script.ScriptException: java.lang.RuntimeException: Oops
   	at org.codehaus.groovy.jsr223.GroovyScriptEngineImpl.eval(GroovyScriptEngineImpl.java:162)
   	at javax.script.AbstractScriptEngine.eval(AbstractScriptEngine.java:233)
   	at org.apache.nifi.processors.script.InterpretedScriptEvaluator.evaluate(InterpretedScriptEvaluator.java:47)
   	at org.apache.nifi.processors.script.ScriptedRouterProcessor$1.process(ScriptedRouterProcessor.java:143)
   	... 16 common frames omitted
   Caused by: javax.script.ScriptException: java.lang.RuntimeException: Oops
   	at org.codehaus.groovy.jsr223.GroovyScriptEngineImpl.eval(GroovyScriptEngineImpl.java:324)
   	at org.codehaus.groovy.jsr223.GroovyScriptEngineImpl.eval(GroovyScriptEngineImpl.java:159)
   	... 19 common frames omitted
   Caused by: java.lang.RuntimeException: Oops
   	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
   	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
   	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
   	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
   	at org.codehaus.groovy.reflection.CachedConstructor.invoke(CachedConstructor.java:83)
   	at org.codehaus.groovy.runtime.callsite.ConstructorSite$ConstructorSiteNoUnwrapNoCoerce.callConstructor(ConstructorSite.java:105)
   	at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCallConstructor(CallSiteArray.java:59)
   	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.callConstructor(AbstractCallSite.java:237)
   	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.callConstructor(AbstractCallSite.java:249)
   	at Script7.run(Script7.groovy:1)
   	at org.codehaus.groovy.jsr223.GroovyScriptEngineImpl.eval(GroovyScriptEngineImpl.java:321)
   	... 20 common frames omitted
   ```
   
   There was no error parsing the incoming FlowFile - it parsed just fine. The error was in evaluating the script against the record. It will be important that we give an accurate error message. And because a FlowFile may have many (often thousands or more) records, it will be much easier to understand what the problem is if the error message indicates which Record was being processed when it failed. For example: "Failed to process Record 529 in FlowFile[...]"
   
   Otherwise, I think all is looking good!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org