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 2020/11/09 19:57:37 UTC

[GitHub] [nifi] MikeThomsen commented on a change in pull request #4638: NIFI-7906: parameterized graph query

MikeThomsen commented on a change in pull request #4638:
URL: https://github.com/apache/nifi/pull/4638#discussion_r520078624



##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")
+                                                    .description("The response object from the graph server")
+                                                    .build();
+
+    public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS, REL_FAILURE, REL_ERRORS, REL_GRAPH
+    )));
+
+    public static final String RECORD_COUNT = "records.count";
+    public static final String GRAPH_OPERATION_TIME = "graph.operations.took";
+    private volatile RecordPathCache recordPathCache;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private GraphClientService clientService;
+    private RecordReaderFactory recordReaderFactory;
+    private RecordSetWriterFactory recordSetWriterFactory;
+    private ObjectMapper mapper = new ObjectMapper();
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+        recordReaderFactory = context.getProperty(READER_SERVICE).asControllerService(RecordReaderFactory.class);
+        recordSetWriterFactory = context.getProperty(WRITER_SERVICE).asControllerService(RecordSetWriterFactory.class);
+        recordPathCache = new RecordPathCache(100);
+    }
+
+    private List<Object> getRecordValue(ProcessContext context, PropertyDescriptor entry, Record record, FlowFile input){
+        String valueRecordPath = context.getProperty(entry.getName()).evaluateAttributeExpressions(input).getValue();
+        final RecordPath recordPath = recordPathCache.getCompiled(valueRecordPath);
+        final RecordPathResult result = recordPath.evaluate(record);
+        return result.getSelectedFields()
+                .filter(fv -> fv.getValue() != null)
+                .map(FieldValue::getValue)
+                .collect( Collectors.toList());
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        FlowFile output = session.create(input);
+        FlowFile graph = session.create(input);

Review comment:
       Doing an all-at-once output here could be very expensive with a large data set, so break this up into a list of flowfiles and do each result to a separate flowfile.

##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")

Review comment:
       Let's set this up to be auto-terminated because many users wouldn't need it.

##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")
+                                                    .description("The response object from the graph server")
+                                                    .build();
+
+    public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS, REL_FAILURE, REL_ERRORS, REL_GRAPH
+    )));
+
+    public static final String RECORD_COUNT = "records.count";
+    public static final String GRAPH_OPERATION_TIME = "graph.operations.took";
+    private volatile RecordPathCache recordPathCache;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private GraphClientService clientService;
+    private RecordReaderFactory recordReaderFactory;
+    private RecordSetWriterFactory recordSetWriterFactory;
+    private ObjectMapper mapper = new ObjectMapper();
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+        recordReaderFactory = context.getProperty(READER_SERVICE).asControllerService(RecordReaderFactory.class);
+        recordSetWriterFactory = context.getProperty(WRITER_SERVICE).asControllerService(RecordSetWriterFactory.class);
+        recordPathCache = new RecordPathCache(100);
+    }
+
+    private List<Object> getRecordValue(ProcessContext context, PropertyDescriptor entry, Record record, FlowFile input){
+        String valueRecordPath = context.getProperty(entry.getName()).evaluateAttributeExpressions(input).getValue();
+        final RecordPath recordPath = recordPathCache.getCompiled(valueRecordPath);
+        final RecordPathResult result = recordPath.evaluate(record);
+        return result.getSelectedFields()
+                .filter(fv -> fv.getValue() != null)
+                .map(FieldValue::getValue)
+                .collect( Collectors.toList());
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        FlowFile output = session.create(input);
+        FlowFile graph = session.create(input);
+        session.getProvenanceReporter().clone(input, graph);
+
+        String recordScript = context.getProperty(SUBMISSION_SCRIPT)
+                .evaluateAttributeExpressions(input)
+                .getValue();
+
+        boolean failed = false;
+        boolean error = false;
+        final AtomicLong errors = new AtomicLong();
+        long delta = 0;
+        Map<String, String> attributes = new HashMap<>();
+        List<Map<String,Object>> graphResponses = new ArrayList<>();
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output);
+             OutputStream graphOutputStream = session.write(graph);
+             RecordReader reader = recordReaderFactory.createRecordReader(attributes, is, -1l, getLogger());
+             RecordSetWriter writer = recordSetWriterFactory.createWriter(getLogger(), reader.getSchema(), os, input.getAttributes());
+        ) {
+            List<PropertyDescriptor> dynamic = context.getProperties()
+                    .keySet().stream()
+                    .filter(PropertyDescriptor::isDynamic)
+                    .collect(Collectors.toList());
+            Record record;
+
+            long start = System.currentTimeMillis();
+            long recordIndex = 0;
+            long batchStart = start;
+            writer.beginRecordSet();
+            while ((record = reader.nextRecord()) != null) {
+                Map<String, Object> dynamicPropertyMap = new HashMap<>();
+                for (PropertyDescriptor entry : dynamic) {
+                        if(!dynamicPropertyMap.containsKey(entry.getName())) {
+                            dynamicPropertyMap.put(entry.getName(), getRecordValue(context, entry, record, input));
+                        }
+                }
+                dynamicPropertyMap.putAll(input.getAttributes());
+                graphResponses.addAll(executeQuery(recordScript, dynamicPropertyMap));
+                recordIndex++;
+                if (recordIndex % 100 == 0) {
+                    long now = System.currentTimeMillis();
+                    batchStart = System.currentTimeMillis();
+                }
+            }
+            try {

Review comment:
       If this fails, the whole thing should fail, so you can take it out of its try-catch.

##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")
+                                                    .description("The response object from the graph server")
+                                                    .build();
+
+    public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS, REL_FAILURE, REL_ERRORS, REL_GRAPH
+    )));
+
+    public static final String RECORD_COUNT = "records.count";
+    public static final String GRAPH_OPERATION_TIME = "graph.operations.took";
+    private volatile RecordPathCache recordPathCache;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private GraphClientService clientService;
+    private RecordReaderFactory recordReaderFactory;
+    private RecordSetWriterFactory recordSetWriterFactory;
+    private ObjectMapper mapper = new ObjectMapper();
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+        recordReaderFactory = context.getProperty(READER_SERVICE).asControllerService(RecordReaderFactory.class);
+        recordSetWriterFactory = context.getProperty(WRITER_SERVICE).asControllerService(RecordSetWriterFactory.class);
+        recordPathCache = new RecordPathCache(100);
+    }
+
+    private List<Object> getRecordValue(ProcessContext context, PropertyDescriptor entry, Record record, FlowFile input){
+        String valueRecordPath = context.getProperty(entry.getName()).evaluateAttributeExpressions(input).getValue();
+        final RecordPath recordPath = recordPathCache.getCompiled(valueRecordPath);
+        final RecordPathResult result = recordPath.evaluate(record);
+        return result.getSelectedFields()
+                .filter(fv -> fv.getValue() != null)
+                .map(FieldValue::getValue)
+                .collect( Collectors.toList());
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        FlowFile output = session.create(input);
+        FlowFile graph = session.create(input);
+        session.getProvenanceReporter().clone(input, graph);
+
+        String recordScript = context.getProperty(SUBMISSION_SCRIPT)
+                .evaluateAttributeExpressions(input)
+                .getValue();
+
+        boolean failed = false;
+        boolean error = false;
+        final AtomicLong errors = new AtomicLong();
+        long delta = 0;
+        Map<String, String> attributes = new HashMap<>();
+        List<Map<String,Object>> graphResponses = new ArrayList<>();
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output);
+             OutputStream graphOutputStream = session.write(graph);
+             RecordReader reader = recordReaderFactory.createRecordReader(attributes, is, -1l, getLogger());
+             RecordSetWriter writer = recordSetWriterFactory.createWriter(getLogger(), reader.getSchema(), os, input.getAttributes());
+        ) {
+            List<PropertyDescriptor> dynamic = context.getProperties()
+                    .keySet().stream()
+                    .filter(PropertyDescriptor::isDynamic)
+                    .collect(Collectors.toList());
+            Record record;
+
+            long start = System.currentTimeMillis();
+            long recordIndex = 0;
+            long batchStart = start;
+            writer.beginRecordSet();
+            while ((record = reader.nextRecord()) != null) {
+                Map<String, Object> dynamicPropertyMap = new HashMap<>();
+                for (PropertyDescriptor entry : dynamic) {
+                        if(!dynamicPropertyMap.containsKey(entry.getName())) {
+                            dynamicPropertyMap.put(entry.getName(), getRecordValue(context, entry, record, input));
+                        }
+                }
+                dynamicPropertyMap.putAll(input.getAttributes());
+                graphResponses.addAll(executeQuery(recordScript, dynamicPropertyMap));
+                recordIndex++;
+                if (recordIndex % 100 == 0) {

Review comment:
       This looks like dead code at this point, so remove it and the `batchStart` declaration.

##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")
+                                                    .description("The response object from the graph server")
+                                                    .build();
+
+    public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS, REL_FAILURE, REL_ERRORS, REL_GRAPH
+    )));
+
+    public static final String RECORD_COUNT = "records.count";
+    public static final String GRAPH_OPERATION_TIME = "graph.operations.took";
+    private volatile RecordPathCache recordPathCache;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private GraphClientService clientService;
+    private RecordReaderFactory recordReaderFactory;
+    private RecordSetWriterFactory recordSetWriterFactory;
+    private ObjectMapper mapper = new ObjectMapper();
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+        recordReaderFactory = context.getProperty(READER_SERVICE).asControllerService(RecordReaderFactory.class);
+        recordSetWriterFactory = context.getProperty(WRITER_SERVICE).asControllerService(RecordSetWriterFactory.class);
+        recordPathCache = new RecordPathCache(100);
+    }
+
+    private List<Object> getRecordValue(ProcessContext context, PropertyDescriptor entry, Record record, FlowFile input){
+        String valueRecordPath = context.getProperty(entry.getName()).evaluateAttributeExpressions(input).getValue();
+        final RecordPath recordPath = recordPathCache.getCompiled(valueRecordPath);
+        final RecordPathResult result = recordPath.evaluate(record);
+        return result.getSelectedFields()
+                .filter(fv -> fv.getValue() != null)
+                .map(FieldValue::getValue)
+                .collect( Collectors.toList());
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        FlowFile output = session.create(input);
+        FlowFile graph = session.create(input);
+        session.getProvenanceReporter().clone(input, graph);
+
+        String recordScript = context.getProperty(SUBMISSION_SCRIPT)
+                .evaluateAttributeExpressions(input)
+                .getValue();
+
+        boolean failed = false;
+        boolean error = false;
+        final AtomicLong errors = new AtomicLong();
+        long delta = 0;
+        Map<String, String> attributes = new HashMap<>();
+        List<Map<String,Object>> graphResponses = new ArrayList<>();
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output);
+             OutputStream graphOutputStream = session.write(graph);
+             RecordReader reader = recordReaderFactory.createRecordReader(attributes, is, -1l, getLogger());
+             RecordSetWriter writer = recordSetWriterFactory.createWriter(getLogger(), reader.getSchema(), os, input.getAttributes());
+        ) {
+            List<PropertyDescriptor> dynamic = context.getProperties()
+                    .keySet().stream()
+                    .filter(PropertyDescriptor::isDynamic)
+                    .collect(Collectors.toList());
+            Record record;
+
+            long start = System.currentTimeMillis();
+            long recordIndex = 0;
+            long batchStart = start;
+            writer.beginRecordSet();
+            while ((record = reader.nextRecord()) != null) {
+                Map<String, Object> dynamicPropertyMap = new HashMap<>();
+                for (PropertyDescriptor entry : dynamic) {
+                        if(!dynamicPropertyMap.containsKey(entry.getName())) {
+                            dynamicPropertyMap.put(entry.getName(), getRecordValue(context, entry, record, input));
+                        }
+                }
+                dynamicPropertyMap.putAll(input.getAttributes());
+                graphResponses.addAll(executeQuery(recordScript, dynamicPropertyMap));
+                recordIndex++;
+                if (recordIndex % 100 == 0) {
+                    long now = System.currentTimeMillis();
+                    batchStart = System.currentTimeMillis();
+                }
+            }
+            try {
+                String graphOutput = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(graphResponses);
+                graphOutputStream.write(graphOutput.getBytes(StandardCharsets.UTF_8));
+                session.getProvenanceReporter().modifyContent(graph);

Review comment:
       Remove this when you make the list-related change so you don't add `List#size()` extraneous provenance events.

##########
File path: nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQueryRecord.java
##########
@@ -0,0 +1,289 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.record.path.util.RecordPathCache;
+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.Record;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+@Tags({"graph, gremlin"})
+@CapabilityDescription("This uses a flowfile as input to perform graph mutations.")
+@WritesAttributes({
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.GRAPH_OPERATION_TIME, description = "The amount of time it took to execute all of the graph operations."),
+        @WritesAttribute(attribute = ExecuteGraphQueryRecord.RECORD_COUNT, description = "The amount of record processed")
+})
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@DynamicProperty(name = "A FlowFile property to be used as a parameter in the graph script",
+        value = "The variable name to be set", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Uses a record path to set a variable as a parameter in the graph script")
+public class ExecuteGraphQueryRecord extends  AbstractGraphExecutor {
+
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("client-service")
+            .displayName("Client Service")
+            .description("The graph client service for connecting to a graph database.")
+            .identifiesControllerService(GraphClientService.class)
+            .addValidator(Validator.VALID)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor READER_SERVICE = new PropertyDescriptor.Builder()
+            .name("reader-service")
+            .displayName("Record Reader")
+            .description("The record reader to use with this processor.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor WRITER_SERVICE = new PropertyDescriptor.Builder()
+            .name("writer-service")
+            .displayName("Failed Record Writer")
+            .description("The record writer to use for writing failed records.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .addValidator(Validator.VALID)
+            .build();
+
+    public static final PropertyDescriptor SUBMISSION_SCRIPT = new PropertyDescriptor.Builder()
+            .name("record-script")
+            .displayName("Graph Record Script")
+            .description("Script to perform the business logic on graph, using flow file attributes and custom properties " +
+                    "as variable-value pairs in its logic.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .dynamic(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+    }
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CLIENT_SERVICE, READER_SERVICE, WRITER_SERVICE, SUBMISSION_SCRIPT
+    ));
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("original")
+                                                    .description("The original flowfile")
+                                                    .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+                                                    .description("Flow files that fail to interact with graph server")
+                                                    .build();
+    public static final Relationship REL_ERRORS = new Relationship.Builder().name("errors")
+                                                    .description("Flow files that error in the response from graph server")
+                                                    .build();
+    public static final Relationship REL_GRAPH = new Relationship.Builder().name("response")
+                                                    .description("The response object from the graph server")
+                                                    .build();
+
+    public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_SUCCESS, REL_FAILURE, REL_ERRORS, REL_GRAPH
+    )));
+
+    public static final String RECORD_COUNT = "records.count";
+    public static final String GRAPH_OPERATION_TIME = "graph.operations.took";
+    private volatile RecordPathCache recordPathCache;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private GraphClientService clientService;
+    private RecordReaderFactory recordReaderFactory;
+    private RecordSetWriterFactory recordSetWriterFactory;
+    private ObjectMapper mapper = new ObjectMapper();
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+        recordReaderFactory = context.getProperty(READER_SERVICE).asControllerService(RecordReaderFactory.class);
+        recordSetWriterFactory = context.getProperty(WRITER_SERVICE).asControllerService(RecordSetWriterFactory.class);
+        recordPathCache = new RecordPathCache(100);
+    }
+
+    private List<Object> getRecordValue(ProcessContext context, PropertyDescriptor entry, Record record, FlowFile input){
+        String valueRecordPath = context.getProperty(entry.getName()).evaluateAttributeExpressions(input).getValue();
+        final RecordPath recordPath = recordPathCache.getCompiled(valueRecordPath);
+        final RecordPathResult result = recordPath.evaluate(record);
+        return result.getSelectedFields()
+                .filter(fv -> fv.getValue() != null)
+                .map(FieldValue::getValue)
+                .collect( Collectors.toList());
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile input = session.get();
+        if ( input == null ) {
+            return;
+        }
+
+        FlowFile output = session.create(input);
+        FlowFile graph = session.create(input);
+        session.getProvenanceReporter().clone(input, graph);
+
+        String recordScript = context.getProperty(SUBMISSION_SCRIPT)
+                .evaluateAttributeExpressions(input)
+                .getValue();
+
+        boolean failed = false;
+        boolean error = false;
+        final AtomicLong errors = new AtomicLong();
+        long delta = 0;
+        Map<String, String> attributes = new HashMap<>();
+        List<Map<String,Object>> graphResponses = new ArrayList<>();
+        try (InputStream is = session.read(input);
+             OutputStream os = session.write(output);
+             OutputStream graphOutputStream = session.write(graph);
+             RecordReader reader = recordReaderFactory.createRecordReader(attributes, is, -1l, getLogger());
+             RecordSetWriter writer = recordSetWriterFactory.createWriter(getLogger(), reader.getSchema(), os, input.getAttributes());
+        ) {
+            List<PropertyDescriptor> dynamic = context.getProperties()
+                    .keySet().stream()
+                    .filter(PropertyDescriptor::isDynamic)
+                    .collect(Collectors.toList());
+            Record record;
+
+            long start = System.currentTimeMillis();
+            long recordIndex = 0;
+            long batchStart = start;
+            writer.beginRecordSet();
+            while ((record = reader.nextRecord()) != null) {
+                Map<String, Object> dynamicPropertyMap = new HashMap<>();
+                for (PropertyDescriptor entry : dynamic) {
+                        if(!dynamicPropertyMap.containsKey(entry.getName())) {
+                            dynamicPropertyMap.put(entry.getName(), getRecordValue(context, entry, record, input));
+                        }
+                }
+                dynamicPropertyMap.putAll(input.getAttributes());
+                graphResponses.addAll(executeQuery(recordScript, dynamicPropertyMap));
+                recordIndex++;
+                if (recordIndex % 100 == 0) {
+                    long now = System.currentTimeMillis();
+                    batchStart = System.currentTimeMillis();
+                }
+            }
+            try {
+                String graphOutput = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(graphResponses);
+                graphOutputStream.write(graphOutput.getBytes(StandardCharsets.UTF_8));
+                session.getProvenanceReporter().modifyContent(graph);
+            } catch ( Exception ex ) {
+                getLogger().error("", ex);
+                error = true;
+            }
+            writer.finishRecordSet();
+            long end = System.currentTimeMillis();
+            delta = (end - start) / 1000;
+            if (getLogger().isDebugEnabled()){
+                getLogger().debug(String.format("Took %s seconds.", delta));
+            }
+        } catch (Exception ex) {
+            getLogger().error("", ex);
+            failed = true;
+        } finally {
+            if (failed) {
+                session.remove(output);

Review comment:
       Removing graph output should happen 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.

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