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/12/15 22:24:14 UTC

[GitHub] [nifi] markap14 opened a new pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

markap14 opened a new pull request #4730:
URL: https://github.com/apache/nifi/pull/4730


   … Minor updates to stateless nifi api to accommodate.
   
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       So the source topic partition is identified by the task index, is that correct?
   My question is, what happens if we start a connector with 3 tasks, it starts working and committing offsets with task ids 0, 1, 2. Then, we reconfigure the connector, and reduce the number of tasks to 2. Now we have task ids 0, 1. What happens with the committed offsets of task no. 2? Can the remaining 2 tasks pick up the work which was originally done by task no. 2?




----------------------------------------------------------------
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 #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   Pushed an update that I think should address all concerns above. Many thanks for the review & guidance @urbandan @heritamas and @in-park!


----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+    private boolean lastTriggerSuccessful = true;
+    private ExecutorService backgroundTriggerExecutor;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+
+        backgroundTriggerExecutor = Executors.newFixedThreadPool(1, r -> {
+            final Thread thread = Executors.defaultThreadFactory().newThread(r);
+            thread.setName("Execute dataflow " + dataflowName);
+            return thread;
+        });
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (!lastTriggerSuccessful) {
+            // When Kafka Connect calls #put, it expects the method to return quickly. If the dataflow should be triggered, it needs to be triggered
+            // in a background thread. When that happens, it's possible that the dataflow could fail, in which case we'd want to rollback and re-deliver
+            // the messages. Because the background thread cannot readily do that, it sets a flag, `lastTriggerSuccessful = false`. We check this here,
+            // so that if a background task failed, we can throw a RetriableException, resulting in the messages being requeued. Because of that, we ensure
+            // that any time that we set lastTriggerSuccessful, we also purge any data in the dataflow, so that it can be redelivered and retried.
+
+            lastTriggerSuccessful = true; // We don't want to throw a RetriableException again.
+            throw new RetriableException("Last attempt to trigger dataflow failed");
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        // If we haven't reached the preferred back size, return.
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow in background thread");
+
+        backgroundTriggerExecutor.submit(this::triggerDataflow);

Review comment:
       As we discussed offline, would be preferable to do the trigger in flush, and only enqueue in put.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +

Review comment:
       If the task should only run in a single instance, the connector should only create a single task configuration.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port. Expected: " + expectedPortName + ". Actual: " + portName);
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents, final Map<String, ?> componentState, final Map<String, ?> partitionMap) {
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final ConnectHeaders headers = new ConnectHeaders();
+        if (headerAttributeNamePattern != null) {
+            // TODO: When we download/create the dataflow, create a hash of it. Then save that state. When we do it next time,
+            //       compare the hash to the last one. If changed, need to trigger connect framework to tell it that the config has changed.
+            //       Would be done via Source/Sink Context.
+            //       Or perhaps we should include the flow JSON itself in the configuration... would require that we string-ify the JSON though. This would be the cleanest, though. Would be optional.
+            //       We can just document that you either include it inline, or you don't make changes to the dataflow; instead, save as a separate dataflow and update task to point to the new one.
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+                    headers.add(headerName, headerValue, Schema.STRING_SCHEMA);
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partitionMap, componentState, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {
+        super.commitRecord(record, metadata);
+
+        final long unacked = unacknowledgedRecords.decrementAndGet();
+        logger.debug("SourceRecord {} committed; number of unacknowledged FlowFiles is now {}", record, unacked);
+
+        if (unacked < 1) {

Review comment:
       Since this is usually invoked from another thread, there can be an edge case where:
   
   - the unacknowledgedRecords is already decremented, making it 0, but the result is still not acknowledged
   - poll is invoked at this point on the task thread, and we trigger the dataflow while the previous result is still not acknowledged.




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       Thanks, I see - then I don't really see the added value of the local state, and seems like a good way of shooting yourself in the leg when configuring the connector




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);

Review comment:
       As discussed offline, it's preferable to only set the latest offset (component state) for the last record of the batch.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", properties);
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get("name");
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {

Review comment:
       That's a very good point. I wanted to revisit this but had forgot about it. Will look into how to integrate this well with NiFi StateProvider mechanism so that we can store this appropriately.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);

Review comment:
       Thanks. Will address.




----------------------------------------------------------------
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 #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   Thanks for sticking with me as I learn the connect API @urbandan and for all of the fantastic feedback!


----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       Yes. So in NiFi, processors can store either 'local' state or 'cluster-wide' state. Typically local state would be used if tailing a file on the local file system or something like that. In that case, it would map to the task id. If you want from 3 tasks to 2, that's fine. The state/progress would be lost so there may be some data duplication. But tailing a file in a local file system wouldn't really be a great use case for connect since the task could be potentially started elsewhere, etc. For the most part local state won't be used, though.
   
   Cluster-wide state would be used if gathering data from S3 or a GCS bucket, for example. In that case, you'd likely only use a single task due to the fact that the protocol itself doesn't offer queuing semantics so it's difficult to scale. But for cases where cluster-wide state is used by the Processor, the state is instead stored here using the `clusterStatePartitionMap`.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+    private boolean lastTriggerSuccessful = true;
+    private ExecutorService backgroundTriggerExecutor;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+
+        backgroundTriggerExecutor = Executors.newFixedThreadPool(1, r -> {
+            final Thread thread = Executors.defaultThreadFactory().newThread(r);
+            thread.setName("Execute dataflow " + dataflowName);
+            return thread;
+        });
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (!lastTriggerSuccessful) {
+            // When Kafka Connect calls #put, it expects the method to return quickly. If the dataflow should be triggered, it needs to be triggered
+            // in a background thread. When that happens, it's possible that the dataflow could fail, in which case we'd want to rollback and re-deliver
+            // the messages. Because the background thread cannot readily do that, it sets a flag, `lastTriggerSuccessful = false`. We check this here,
+            // so that if a background task failed, we can throw a RetriableException, resulting in the messages being requeued. Because of that, we ensure
+            // that any time that we set lastTriggerSuccessful, we also purge any data in the dataflow, so that it can be redelivered and retried.
+
+            lastTriggerSuccessful = true; // We don't want to throw a RetriableException again.
+            throw new RetriableException("Last attempt to trigger dataflow failed");
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        // If we haven't reached the preferred back size, return.
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow in background thread");
+
+        backgroundTriggerExecutor.submit(this::triggerDataflow);

Review comment:
       Will do.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";

Review comment:
       Fair enough. I actually think pulling it out into a variable makes sense, rather than trying to simplify to a single return. But it's good to avoid the repetition. 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.

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



[GitHub] [nifi] urbandan commented on pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   @markap14 Sorry I haven't managed to finish reviewing yet, but I think there are a few issues we should discuss - will try to complete the review ASAP


----------------------------------------------------------------
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 #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   Thanks @exceptionfactory for reviewing. Pushed a new commit that I believe addresses all feedback 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.

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



[GitHub] [nifi] markap14 commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       Yes. So in NiFi, processors can store either 'local' state or 'cluster-wide' state. Typically local state would be used if tailing a file on the local file system or something like that. In that case, it would map to the task id. If you want from 3 tasks to 2, that's fine. The state/progress would be lost so there may be some data duplication. But tailing a file in a local file system wouldn't really be a great use case for connect since the task could be potentially started elsewhere, etc. For the most part local state won't be used, though.
   
   Cluster-wide state would be used if gathering data from S3 or a GCS bucket, for example. In that case, you'd likely only use a single task due to the fact that the protocol itself doesn't offer queuing semantics so it's difficult to scale. But for cases where cluster-wide state is used by the Processor, the state is instead stored here using the `clusterStatePartitionMap`.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port. Expected: " + expectedPortName + ". Actual: " + portName);
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents, final Map<String, ?> componentState, final Map<String, ?> partitionMap) {
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final ConnectHeaders headers = new ConnectHeaders();
+        if (headerAttributeNamePattern != null) {
+            // TODO: When we download/create the dataflow, create a hash of it. Then save that state. When we do it next time,
+            //       compare the hash to the last one. If changed, need to trigger connect framework to tell it that the config has changed.
+            //       Would be done via Source/Sink Context.
+            //       Or perhaps we should include the flow JSON itself in the configuration... would require that we string-ify the JSON though. This would be the cleanest, though. Would be optional.
+            //       We can just document that you either include it inline, or you don't make changes to the dataflow; instead, save as a separate dataflow and update task to point to the new one.
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+                    headers.add(headerName, headerValue, Schema.STRING_SCHEMA);
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partitionMap, componentState, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {
+        super.commitRecord(record, metadata);
+
+        final long unacked = unacknowledgedRecords.decrementAndGet();
+        logger.debug("SourceRecord {} committed; number of unacknowledged FlowFiles is now {}", record, unacked);
+
+        if (unacked < 1) {

Review comment:
       I think this scenario is okay. When that happens, the call to `dataflow.trigger()` will happen. This will return a `DataflowTriggerResult` immediately, but the call to `DataflowTriggerResult.getResult()` will block until the dataflow completes in the background. But the dataflow won't actually run in the background until the previous one is acknowledged. So the second triggering of the dataflow doesn't actually happen until the previous one has been acknowledged.
   
   Unless i'm overlooking something on the connect side that makes this problematic?




----------------------------------------------------------------
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 closed pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   


----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;
+        if (headerAttributeNamePattern == null) {
+            headers = Collections.emptyList();
+        } else {
+            headers = new ArrayList<>();
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+
+                    headers.add(new StatelessNiFiKafkaHeader(headerName, headerValue));
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partition, sourceOffset, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {

Review comment:
       Stateless framework is going to prevent a second invocation until the previous one completes. But I can easily determine if the previous invocation is still working and if so, return `null` from poll().




----------------------------------------------------------------
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] joewitt commented on pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   awesome work and discussion and review here.  this is going to be awesome.
   
   I'm a +1 based on watching this engagement and reviewing the changes.  
   
   mark please self merge as I cannot get to that part at this time.


----------------------------------------------------------------
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 #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   Thanks for sticking with me as I learn the connect API @urbandan and for all of the fantastic feedback!


----------------------------------------------------------------
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] heritamas commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", properties);
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get("name");
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {

Review comment:
       How exactly do we track the progress of copying data? AFAIK, in the Connect framework the usually abstracted away concepts of topic-partition and offset is used for that. If connector is rebalanced, the abstract topic-partition and offset is used to determine where the copy should continue from. Here partition and offset seem to be abandoned, or am I missing something? 




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);

Review comment:
       In general, backoff should be done by throwing a RetriableException - this method isn't supposed to block.
   org.apache.kafka.connect.sink.SinkTaskContext#timeout can be used to define the delay for the retry.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);

Review comment:
       If the method threw a RetriableException previously, the framework will repeat the previous call with the same records - those records are enqueued here repeatedly. Is this correct from the dataflow's point of view? What happens with the previously enqueued, but not yet triggered records?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);

Review comment:
       This will block put, but put is expected to send async. I think the trigger should be reworked to be done on a background thread to not block the thread of put.
   Not completely sure about this, but also think that the retry of the trigger should be separate from the retry of put. (e.g. put should be retried if the buffer in the dataflow is full, trigger should be retried if the external system had issues).

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkConnector.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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.sink.SinkConnector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSinkConnector extends SinkConnector {
+    static final String INPUT_PORT_NAME = "input.port";
+    static final String FAILURE_PORTS = "failure.ports";
+    static final String HEADERS_AS_ATTRIBUTES_REGEX = "headers.as.attributes.regex";
+    static final String HEADER_ATTRIBUTE_NAME_PREFIX = "attribute.prefix";
+    static final String BATCH_SIZE_COUNT = "batch.size.count";
+    static final String BATCH_SIZE_BYTES = "batch.size.bytes";
+
+    private Map<String, String> properties;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {
+        this.properties = new HashMap<>(this.properties);
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return StatelessNiFiSinkTask.class;
+    }
+
+    @Override
+    public List<Map<String, String>> taskConfigs(final int maxTasks) {
+        final List<Map<String, String>> configs = new ArrayList<>();
+        for (int i=0; i < maxTasks; i++) {
+            configs.add(new HashMap<>(properties));
+        }
+
+        return configs;
+    }
+
+    @Override
+    public void stop() {
+    }
+
+    @Override
+    public ConfigDef config() {
+        final ConfigDef configDef = new ConfigDef();
+        StatelessKafkaConnectorUtil.addCommonConfigElements(configDef);
+
+        configDef.define(INPUT_PORT_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "The name of the Input Port to push data to");
+        configDef.define(HEADERS_AS_ATTRIBUTES_REGEX, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "A regular expression to evaluate against Kafka message header keys. Any message " +
+            "header whose key matches the regular expression will be added to the FlowFile as an attribute. The name of the attribute will match the header key (with an optional prefix, as " +
+            "defined by the attribute.prefix configuration) and the header value will be added as the attribute value.");
+        configDef.define(HEADER_ATTRIBUTE_NAME_PREFIX, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "A prefix to add to the key of each header that matches the headers.as.attributes.regex Regular Expression. For example, if a header has the ke MyHeader and a value of " +

Review comment:
       Typo: "if a header has the **key**"

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+                if (resultOptional.isPresent()) {
+                    final TriggerResult result = resultOptional.get();
+
+                    if (result.isSuccessful()) {
+                        // Verify that data was only transferred to the expected Input Port
+                        verifyOutputPortContents(trigger, result);
+
+                        // Acknowledge the data so that the session can be committed
+                        result.acknowledge();

Review comment:
       After this point, org.apache.kafka.connect.sink.SinkTaskContext#requestCommit should be invoked to try to minimize reprocessing.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;
+        if (headerAttributeNamePattern == null) {
+            headers = Collections.emptyList();
+        } else {
+            headers = new ArrayList<>();
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+
+                    headers.add(new StatelessNiFiKafkaHeader(headerName, headerValue));
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partition, sourceOffset, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {

Review comment:
       This method is usually called by a producer network thread when the record batch was ACK'd by the broker.
   It is possible that poll is called multiple times before the commitRecord is invoked with the records returned by the first call of poll. Because of this, the result cached in triggerResult might be overwritten, and never get acknowledged.
   The task might need to track multiple TriggerResults to be able to meaningfully acknowledge them.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;

Review comment:
       Instead of a list with a custom Header implementation, org.apache.kafka.connect.header.ConnectHeaders can be used. That is capable of instantiating the built-in ConnectHeader class, and also implements Iterable<Header>




----------------------------------------------------------------
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] joewitt commented on pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   awesome work and discussion and review here.  this is going to be awesome.
   
   I'm a +1 based on watching this engagement and reviewing the changes.  
   
   mark please self merge as I cannot get to that part at this time.


----------------------------------------------------------------
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 closed pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   


----------------------------------------------------------------
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] in-park commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

Posted by GitBox <gi...@apache.org>.
in-park commented on a change in pull request #4730:
URL: https://github.com/apache/nifi/pull/4730#discussion_r554673185



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topic.name";

Review comment:
        To be consistent with SinkConnector.TOPICS_CONFIG and to allow SMM UI to be able to pick up the topic names could this be changed to "topics" instead? 




----------------------------------------------------------------
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] heritamas commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", properties);
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get("name");
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {

Review comment:
       How exactly do we track the progress of copying data? AFAIK, in the Connect framework the usually abstracted away concepts of topic-partition and offset is used for that. If connector is rebalanced, the abstract topic-partition and offset is used to determine where the copy should continue from. Here partition and offset seem to be abandoned, or am I missing something? 




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);

Review comment:
       In general, backoff should be done by throwing a RetriableException - this method isn't supposed to block.
   org.apache.kafka.connect.sink.SinkTaskContext#timeout can be used to define the delay for the retry.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);

Review comment:
       If the method threw a RetriableException previously, the framework will repeat the previous call with the same records - those records are enqueued here repeatedly. Is this correct from the dataflow's point of view? What happens with the previously enqueued, but not yet triggered records?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);

Review comment:
       This will block put, but put is expected to send async. I think the trigger should be reworked to be done on a background thread to not block the thread of put.
   Not completely sure about this, but also think that the retry of the trigger should be separate from the retry of put. (e.g. put should be retried if the buffer in the dataflow is full, trigger should be retried if the external system had issues).

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkConnector.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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.sink.SinkConnector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSinkConnector extends SinkConnector {
+    static final String INPUT_PORT_NAME = "input.port";
+    static final String FAILURE_PORTS = "failure.ports";
+    static final String HEADERS_AS_ATTRIBUTES_REGEX = "headers.as.attributes.regex";
+    static final String HEADER_ATTRIBUTE_NAME_PREFIX = "attribute.prefix";
+    static final String BATCH_SIZE_COUNT = "batch.size.count";
+    static final String BATCH_SIZE_BYTES = "batch.size.bytes";
+
+    private Map<String, String> properties;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {
+        this.properties = new HashMap<>(this.properties);
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return StatelessNiFiSinkTask.class;
+    }
+
+    @Override
+    public List<Map<String, String>> taskConfigs(final int maxTasks) {
+        final List<Map<String, String>> configs = new ArrayList<>();
+        for (int i=0; i < maxTasks; i++) {
+            configs.add(new HashMap<>(properties));
+        }
+
+        return configs;
+    }
+
+    @Override
+    public void stop() {
+    }
+
+    @Override
+    public ConfigDef config() {
+        final ConfigDef configDef = new ConfigDef();
+        StatelessKafkaConnectorUtil.addCommonConfigElements(configDef);
+
+        configDef.define(INPUT_PORT_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "The name of the Input Port to push data to");
+        configDef.define(HEADERS_AS_ATTRIBUTES_REGEX, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "A regular expression to evaluate against Kafka message header keys. Any message " +
+            "header whose key matches the regular expression will be added to the FlowFile as an attribute. The name of the attribute will match the header key (with an optional prefix, as " +
+            "defined by the attribute.prefix configuration) and the header value will be added as the attribute value.");
+        configDef.define(HEADER_ATTRIBUTE_NAME_PREFIX, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "A prefix to add to the key of each header that matches the headers.as.attributes.regex Regular Expression. For example, if a header has the ke MyHeader and a value of " +

Review comment:
       Typo: "if a header has the **key**"

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+                if (resultOptional.isPresent()) {
+                    final TriggerResult result = resultOptional.get();
+
+                    if (result.isSuccessful()) {
+                        // Verify that data was only transferred to the expected Input Port
+                        verifyOutputPortContents(trigger, result);
+
+                        // Acknowledge the data so that the session can be committed
+                        result.acknowledge();

Review comment:
       After this point, org.apache.kafka.connect.sink.SinkTaskContext#requestCommit should be invoked to try to minimize reprocessing.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;
+        if (headerAttributeNamePattern == null) {
+            headers = Collections.emptyList();
+        } else {
+            headers = new ArrayList<>();
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+
+                    headers.add(new StatelessNiFiKafkaHeader(headerName, headerValue));
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partition, sourceOffset, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {

Review comment:
       This method is usually called by a producer network thread when the record batch was ACK'd by the broker.
   It is possible that poll is called multiple times before the commitRecord is invoked with the records returned by the first call of poll. Because of this, the result cached in triggerResult might be overwritten, and never get acknowledged.
   The task might need to track multiple TriggerResults to be able to meaningfully acknowledge them.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;

Review comment:
       Instead of a list with a custom Header implementation, org.apache.kafka.connect.header.ConnectHeaders can be used. That is capable of instantiating the built-in ConnectHeader class, and also implements Iterable<Header>

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topic.name";
+
+    static final String TOPIC_NAME_ATTRIBUTE = "topic.name.attribute";
+    static final String KEY_ATTRIBUTE = "key.attribute";
+    static final String HEADER_REGEX = "header.attribute.regex";
+
+    private Map<String, String> properties;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {
+        this.properties = new HashMap<>(this.properties);
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return StatelessNiFiSourceTask.class;
+    }
+
+    @Override
+    public List<Map<String, String>> taskConfigs(final int maxTasks) {
+        final List<Map<String, String>> configs = new ArrayList<>();
+        for (int i=0; i < maxTasks; i++) {
+            configs.add(new HashMap<>(properties));

Review comment:
       At this phase, the connector has to have some information about the input data, and partition that between the created tasks. Not sure how hard it is, or if the stateless nifi flows can coordinate under the hood.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", properties);

Review comment:
       Only thing that should be sensitive is parameters. Will update it to avoid logging any parameters.




----------------------------------------------------------------
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] exceptionfactory commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";

Review comment:
       The method could be refactored to declare this string as the default return value, updated when the Manifest `Implementation-Version` is available, and then have a single return.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";
+    }
+
+    public static StatelessDataflow createDataflow(final Map<String, String> properties) {
+        final StatelessEngineConfiguration engineConfiguration = createEngineConfiguration(properties);
+        final String configuredFlowSnapshot = properties.get(FLOW_SNAPSHOT);
+
+        final List<ParameterOverride> parameterOverrides = parseParameterOverrides(properties);
+        final String dataflowName = properties.get("name");
+
+        final DataflowDefinition<?> dataflowDefinition;
+        final StatelessBootstrap bootstrap;
+        try {
+            final Map<String, String> dataflowDefinitionProperties = new HashMap<>();
+
+            if (configuredFlowSnapshot.startsWith("http://") || configuredFlowSnapshot.startsWith("https://")) {
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.url", configuredFlowSnapshot);

Review comment:
       Should this property key be declared as a static variable?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", properties);
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get("name");

Review comment:
       The `name` property is used in several places and could be declared as a static variable.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";
+    }
+
+    public static StatelessDataflow createDataflow(final Map<String, String> properties) {
+        final StatelessEngineConfiguration engineConfiguration = createEngineConfiguration(properties);
+        final String configuredFlowSnapshot = properties.get(FLOW_SNAPSHOT);
+
+        final List<ParameterOverride> parameterOverrides = parseParameterOverrides(properties);
+        final String dataflowName = properties.get("name");
+
+        final DataflowDefinition<?> dataflowDefinition;
+        final StatelessBootstrap bootstrap;
+        try {
+            final Map<String, String> dataflowDefinitionProperties = new HashMap<>();
+
+            if (configuredFlowSnapshot.startsWith("http://") || configuredFlowSnapshot.startsWith("https://")) {
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.url", configuredFlowSnapshot);
+            } else {
+                final File flowSnapshotFile = new File(configuredFlowSnapshot);
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.file", flowSnapshotFile.getAbsolutePath());
+            }
+
+            dataflowDefinitionProperties.put("nifi.stateless.flow.name", dataflowName);
+
+            MDC.setContextMap(Collections.singletonMap("dataflow", dataflowName));
+
+            // Use a Write Lock to ensure that only a single thread is calling StatelessBootstrap.bootstrap().
+            // We do this because the bootstrap() method will expand all NAR files into the working directory.
+            // If we have multiple Connector instances, or multiple tasks, we don't want several threads all
+            // unpacking NARs at the same time, as it could potentially result in the working directory becoming corrupted.
+            unpackNarLock.lock();
+            try {
+                bootstrap = StatelessBootstrap.bootstrap(engineConfiguration, StatelessNiFiSourceTask.class.getClassLoader());
+            } finally {
+                unpackNarLock.unlock();
+            }
+
+            dataflowDefinition = bootstrap.parseDataflowDefinition(dataflowDefinitionProperties);
+            return bootstrap.createDataflow(dataflowDefinition, parameterOverrides);
+        } catch (final Exception e) {
+            throw new RuntimeException("Failed to bootstrap Stateless NiFi Engine", e);
+        }
+    }
+
+    private static List<ParameterOverride> parseParameterOverrides(final Map<String, String> properties) {
+        final List<ParameterOverride> parameterOverrides = new ArrayList<>();
+
+        for (final Map.Entry<String, String> entry : properties.entrySet()) {
+            final String parameterValue = entry.getValue();
+
+            ParameterOverride parameterOverride = null;
+            final Matcher matcher = PARAMETER_WITH_CONTEXT_PATTERN.matcher(entry.getKey());
+            if (matcher.matches()) {
+                final String contextName = matcher.group(1);
+                final String parameterName = matcher.group(2);
+                parameterOverride = new ParameterOverride(contextName, parameterName, parameterValue);
+            } else {
+                final Matcher noContextMatcher = PARAMETER_WITHOUT_CONTEXT_PATTERN.matcher(entry.getKey());
+                if (noContextMatcher.matches()) {
+                    final String parameterName = noContextMatcher.group(1);
+                    parameterOverride = new ParameterOverride(parameterName, parameterValue);
+                }
+            }
+
+            if (parameterOverride != null) {
+                parameterOverrides.add(parameterOverride);
+            }
+        }
+
+        return parameterOverrides;
+    }
+
+    private static StatelessEngineConfiguration createEngineConfiguration(final Map<String, String> properties) {
+        final File narDirectory;
+        final String narDirectoryFilename = properties.get(NAR_DIRECTORY);
+        if (narDirectoryFilename == null) {
+            narDirectory = detectNarDirectory();
+        } else {
+            narDirectory = new File(narDirectoryFilename);
+        }
+
+        final File workingDirectory;
+        final String workingDirectoryFilename = properties.get(WORKING_DIRECTORY);
+        if (workingDirectoryFilename == null) {
+            workingDirectory = DEFAULT_WORKING_DIRECTORY;
+        } else {
+            workingDirectory = new File(workingDirectoryFilename);
+        }
+
+        final SslContextDefinition sslContextDefinition = createSslContextDefinition(properties);
+
+        final StatelessEngineConfiguration engineConfiguration = new StatelessEngineConfiguration() {
+            @Override
+            public File getWorkingDirectory() {
+                return workingDirectory;
+            }
+
+            @Override
+            public File getNarDirectory() {
+                return narDirectory;
+            }
+
+            @Override
+            public File getKrb5File() {
+                return new File(properties.getOrDefault(KRB5_FILE, DEFAULT_KRB5_FILE));
+            }
+
+            @Override
+            public SslContextDefinition getSslContext() {
+                return sslContextDefinition;
+            }
+
+            @Override
+            public String getSensitivePropsKey() {
+                return "nifi-stateless";
+            }
+
+            @Override
+            public List<ExtensionClientDefinition> getExtensionClients() {
+                final List<ExtensionClientDefinition> extensionClientDefinitions = new ArrayList<>();
+
+                final String nexusBaseUrl = properties.get(NEXUS_BASE_URL);
+                if (nexusBaseUrl != null) {
+                    final ExtensionClientDefinition definition = new ExtensionClientDefinition();
+                    definition.setUseSslContext(false);
+                    definition.setExtensionClientType("nexus");
+                    definition.setCommsTimeout("30 secs");
+                    definition.setBaseUrl(nexusBaseUrl);
+                    extensionClientDefinitions.add(definition);
+                }
+
+                return extensionClientDefinitions;
+            }
+        };
+
+        return engineConfiguration;
+    }
+
+    private static SslContextDefinition createSslContextDefinition(final Map<String, String> properties) {
+        final String truststoreFile = properties.get(TRUSTSTORE_FILE);
+        if (truststoreFile == null || truststoreFile.trim().isEmpty()) {
+            return null;
+        }
+
+        final SslContextDefinition sslContextDefinition;
+        sslContextDefinition = new SslContextDefinition();
+        sslContextDefinition.setTruststoreFile(truststoreFile);
+        sslContextDefinition.setTruststorePass(properties.get(TRUSTSTORE_PASSWORD));
+        sslContextDefinition.setTruststoreType(properties.get(TRUSTSTORE_TYPE));
+
+        final String keystoreFile = properties.get(KEYSTORE_FILE);
+        if (keystoreFile != null && !keystoreFile.trim().isEmpty()) {
+            sslContextDefinition.setKeystoreFile(keystoreFile);
+            sslContextDefinition.setKeystoreType(properties.get(KEYSTORE_TYPE));
+
+            final String keystorePass = properties.get(KEYSTORE_PASSWORD);
+            sslContextDefinition.setKeystorePass(keystorePass);
+
+            final String explicitKeyPass = properties.get(KEY_PASSWORD);
+            final String keyPass = (explicitKeyPass == null || explicitKeyPass.trim().isEmpty()) ? keystorePass : explicitKeyPass;
+            sslContextDefinition.setKeyPass(keyPass);
+        }
+
+        return sslContextDefinition;
+    }
+
+    private static URLClassLoader getConnectClassLoader() {
+        final ClassLoader classLoader = StatelessKafkaConnectorUtil.class.getClassLoader();
+        if (!(classLoader instanceof URLClassLoader)) {
+            throw new IllegalStateException("No configuration value was set for the " + NAR_DIRECTORY + " configuration property, and was unable to determine the NAR directory automatically");
+        }
+
+        return (URLClassLoader) classLoader;
+    }
+
+    private static File detectBootstrapJar() {
+        final URLClassLoader urlClassLoader = getConnectClassLoader();
+        for (final URL url : urlClassLoader.getURLs()) {
+            final String artifactFilename = url.getFile();
+            if (artifactFilename == null) {
+                continue;
+            }
+
+            final File artifactFile = new File(artifactFilename);
+            if (STATELESS_BOOTSTRAP_FILE_PATTERN.matcher(artifactFile.getName()).matches()) {
+                return artifactFile;
+            }
+        }
+
+        return null;
+    }
+
+    private static File detectNarDirectory() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            final URLClassLoader urlClassLoader = getConnectClassLoader();
+            logger.error("ClassLoader that loaded Stateless Kafka Connector did not contain nifi-stateless-bootstrap. URLs that were present: {}", Arrays.asList(urlClassLoader.getURLs()));
+            throw new IllegalStateException("No configuration value was set for the " + NAR_DIRECTORY + " configuration property, and was unable to determine the NAR directory automatically");

Review comment:
       This message appears to be the same as the one used in `getConnectClassLoader`.  The message could be declared once, or this message could be adjusted.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";
+    }
+
+    public static StatelessDataflow createDataflow(final Map<String, String> properties) {
+        final StatelessEngineConfiguration engineConfiguration = createEngineConfiguration(properties);
+        final String configuredFlowSnapshot = properties.get(FLOW_SNAPSHOT);
+
+        final List<ParameterOverride> parameterOverrides = parseParameterOverrides(properties);
+        final String dataflowName = properties.get("name");
+
+        final DataflowDefinition<?> dataflowDefinition;
+        final StatelessBootstrap bootstrap;
+        try {
+            final Map<String, String> dataflowDefinitionProperties = new HashMap<>();
+
+            if (configuredFlowSnapshot.startsWith("http://") || configuredFlowSnapshot.startsWith("https://")) {
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.url", configuredFlowSnapshot);
+            } else {
+                final File flowSnapshotFile = new File(configuredFlowSnapshot);
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.file", flowSnapshotFile.getAbsolutePath());
+            }
+
+            dataflowDefinitionProperties.put("nifi.stateless.flow.name", dataflowName);
+
+            MDC.setContextMap(Collections.singletonMap("dataflow", dataflowName));
+
+            // Use a Write Lock to ensure that only a single thread is calling StatelessBootstrap.bootstrap().
+            // We do this because the bootstrap() method will expand all NAR files into the working directory.
+            // If we have multiple Connector instances, or multiple tasks, we don't want several threads all
+            // unpacking NARs at the same time, as it could potentially result in the working directory becoming corrupted.
+            unpackNarLock.lock();
+            try {
+                bootstrap = StatelessBootstrap.bootstrap(engineConfiguration, StatelessNiFiSourceTask.class.getClassLoader());
+            } finally {
+                unpackNarLock.unlock();
+            }
+
+            dataflowDefinition = bootstrap.parseDataflowDefinition(dataflowDefinitionProperties);
+            return bootstrap.createDataflow(dataflowDefinition, parameterOverrides);
+        } catch (final Exception e) {
+            throw new RuntimeException("Failed to bootstrap Stateless NiFi Engine", e);
+        }
+    }
+
+    private static List<ParameterOverride> parseParameterOverrides(final Map<String, String> properties) {
+        final List<ParameterOverride> parameterOverrides = new ArrayList<>();
+
+        for (final Map.Entry<String, String> entry : properties.entrySet()) {
+            final String parameterValue = entry.getValue();
+
+            ParameterOverride parameterOverride = null;
+            final Matcher matcher = PARAMETER_WITH_CONTEXT_PATTERN.matcher(entry.getKey());
+            if (matcher.matches()) {
+                final String contextName = matcher.group(1);
+                final String parameterName = matcher.group(2);
+                parameterOverride = new ParameterOverride(contextName, parameterName, parameterValue);
+            } else {
+                final Matcher noContextMatcher = PARAMETER_WITHOUT_CONTEXT_PATTERN.matcher(entry.getKey());
+                if (noContextMatcher.matches()) {
+                    final String parameterName = noContextMatcher.group(1);
+                    parameterOverride = new ParameterOverride(parameterName, parameterValue);
+                }
+            }
+
+            if (parameterOverride != null) {
+                parameterOverrides.add(parameterOverride);
+            }
+        }
+
+        return parameterOverrides;
+    }
+
+    private static StatelessEngineConfiguration createEngineConfiguration(final Map<String, String> properties) {
+        final File narDirectory;
+        final String narDirectoryFilename = properties.get(NAR_DIRECTORY);
+        if (narDirectoryFilename == null) {
+            narDirectory = detectNarDirectory();
+        } else {
+            narDirectory = new File(narDirectoryFilename);
+        }
+
+        final File workingDirectory;
+        final String workingDirectoryFilename = properties.get(WORKING_DIRECTORY);
+        if (workingDirectoryFilename == null) {
+            workingDirectory = DEFAULT_WORKING_DIRECTORY;
+        } else {
+            workingDirectory = new File(workingDirectoryFilename);
+        }
+
+        final SslContextDefinition sslContextDefinition = createSslContextDefinition(properties);
+
+        final StatelessEngineConfiguration engineConfiguration = new StatelessEngineConfiguration() {
+            @Override
+            public File getWorkingDirectory() {
+                return workingDirectory;
+            }
+
+            @Override
+            public File getNarDirectory() {
+                return narDirectory;
+            }
+
+            @Override
+            public File getKrb5File() {
+                return new File(properties.getOrDefault(KRB5_FILE, DEFAULT_KRB5_FILE));
+            }
+
+            @Override
+            public SslContextDefinition getSslContext() {
+                return sslContextDefinition;
+            }
+
+            @Override
+            public String getSensitivePropsKey() {
+                return "nifi-stateless";

Review comment:
       Should it be possible to override this value using configuration properties?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", properties);

Review comment:
       See comment on logging properties for the Sink Task.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", properties);
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get("name");
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get("name");
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+                if (resultOptional.isPresent()) {
+                    final TriggerResult result = resultOptional.get();
+
+                    if (result.isSuccessful()) {
+                        // Verify that data was only transferred to the expected Input Port
+                        verifyOutputPortContents(trigger, result);
+
+                        // Acknowledge the data so that the session can be committed
+                        result.acknowledge();
+                    } else {
+                        logger.error("Dataflow {} failed to execute properly", dataflowName, result.getFailureCause().orElse(null));
+                        trigger.cancel();
+                        throw new RetriableException("Dataflow failed to execute properly", result.getFailureCause().orElse(null));
+                    }
+                } else {
+                    trigger.cancel();
+                    throw new RetriableException("Timed out waiting for the dataflow to complete");
+                }
+            } catch (final InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting for dataflow to complete", e);
+            }
+        }
+
+        final long nanos = System.nanoTime() - start;
+        logger.debug("Ran dataflow with {} messages ({}) in {} nanos", queueSize.getObjectCount(), FormatUtils.formatDataSize(queueSize.getByteCount()), nanos);
+    }
+
+    private void verifyOutputPortContents(final DataflowTrigger trigger, final TriggerResult result) {
+        for (final String failurePort : failurePortNames) {
+            final List<FlowFile> flowFiles = result.getOutputFlowFiles(failurePort);
+            if (flowFiles != null && !flowFiles.isEmpty()) {
+                logger.error("Dataflow transferred FlowFiles to Port {}, which is configured as a Failure Port. Rolling back session.", failurePort);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to Failure Port " + failurePort);
+            }
+        }
+    }
+
+    @Override
+    public void flush(final Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
+        super.flush(currentOffsets);
+
+        if (queueSize != null && queueSize.getObjectCount() > 0) {
+            triggerDataflow();
+        }
+    }
+
+    private byte[] getContents(final Object value) {
+        if (value == null) {
+            return new byte[0];
+        }
+        if (value instanceof String) {
+            return ((String) value).getBytes(StandardCharsets.UTF_8);
+        }
+        if (value instanceof byte[]) {
+            return (byte[]) value;
+        }
+
+        throw new IllegalArgumentException("Unsupported message type: the Message value was " + value + " but was expected to be a byte array or a String");
+    }
+
+    private Map<String, String> createAttributes(final SinkRecord record) {
+        final Map<String, String> attributes = new HashMap<>(8);

Review comment:
       Is there a particular reason for declaring an initial size for the HashMap?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", properties);

Review comment:
       Logging all properties will include store passwords and other potentially sensitive parameters.  Perhaps logging a subset of the standard parameters would be a safer approach to avoid writing sensitive information to logs.  Otherwise, some type of filtering should be implemented prior to passing properties for logging.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/config/PropertiesFileFlowDefinitionParser.java
##########
@@ -262,40 +282,93 @@ private void warnOnWhitespace(final Map<String, String> properties, final String
         return properties;
     }
 
-    private VersionedFlowSnapshot fetchVersionedFlowSnapshot(final Map<String, String> properties, final File propertiesFile, final SSLContext sslContext)
+    private VersionedFlowSnapshot fetchVersionedFlowSnapshot(final Map<String, String> properties, final SslContextDefinition sslContextDefinition)
         throws IOException, StatelessConfigurationException {
 
         final String flowSnapshotFilename = properties.get(FLOW_SNAPSHOT_FILE_KEY);
-        if (flowSnapshotFilename == null || flowSnapshotFilename.trim().isEmpty()) {
-            final String registryUrl = properties.get(REGISTRY_URL_KEY);
-            final String bucketId = properties.get(BUCKET_ID_KEY);
-            final String flowId = properties.get(FLOW_ID_KEY);
-            final String flowVersionValue = properties.get(FLOW_VERSION_KEY);
-            final Integer flowVersion;
+        if (flowSnapshotFilename != null && !flowSnapshotFilename.trim().isEmpty()) {
+            final File flowSnapshotFile = new File(flowSnapshotFilename.trim());
             try {
-                flowVersion = flowVersionValue == null || flowVersionValue.trim().isEmpty() ? null : Integer.parseInt(flowVersionValue);
-            } catch (final NumberFormatException nfe) {
-                throw new StatelessConfigurationException("The " + FLOW_VERSION_KEY + " property in " + propertiesFile.getAbsolutePath()
-                    + " was expected to contain a number but had a value of " + flowVersionValue);
+                return readVersionedFlowSnapshot(flowSnapshotFile);
+            } catch (final Exception e) {
+                throw new IOException("Configuration indicates that the flow to run is located at " + flowSnapshotFilename
+                    + " but failed to load dataflow from that location", e);
             }
+        }
 
-            if (registryUrl == null || bucketId == null || flowId == null) {
-                throw new IllegalArgumentException("Specified configuration file " + propertiesFile + " does not provide the filename of the flow to run or the registryUrl, bucketId, and flowId.");
-            }
+        final String flowSnapshotUrl = properties.get(FLOW_SNAPSHOT_URL_KEY);
+        if (flowSnapshotUrl != null && !flowSnapshotUrl.trim().isEmpty()) {
+            final String useSslPropertyValue = properties.get(FLOW_SNAPSHOT_URL_USE_SSLCONTEXT_KEY);
+            final boolean useSsl = Boolean.parseBoolean(useSslPropertyValue);
 
             try {
-                return fetchFlowFromRegistry(registryUrl, bucketId, flowId, flowVersion, sslContext);
-            } catch (final NiFiRegistryException e) {
-                throw new StatelessConfigurationException("Could not fetch flow from Registry", e);
+                return fetchFlowFromUrl(flowSnapshotUrl, useSsl ? sslContextDefinition : null);
+            } catch (final Exception e) {
+                throw new StatelessConfigurationException("Could not fetch flow from URL", e);
             }
         }
 
-        final File flowSnapshotFile = new File(flowSnapshotFilename);
+        // Try downloading flow from registry
+        final String registryUrl = properties.get(REGISTRY_URL_KEY);
+        final String bucketId = properties.get(BUCKET_ID_KEY);
+        final String flowId = properties.get(FLOW_ID_KEY);
+        final String flowVersionValue = properties.get(FLOW_VERSION_KEY);
+        final Integer flowVersion;
         try {
-            return readVersionedFlowSnapshot(flowSnapshotFile);
-        } catch (final Exception e) {
-            throw new IOException("Specified configuration file " + propertiesFile + " indicates that the flow to run is located at " + flowSnapshotFilename
-                + " but failed to load dataflow from that location", e);
+            flowVersion = flowVersionValue == null || flowVersionValue.trim().isEmpty() ? null : Integer.parseInt(flowVersionValue);
+        } catch (final NumberFormatException nfe) {
+            throw new StatelessConfigurationException("The " + FLOW_VERSION_KEY + " property was expected to contain a number but had a value of " + flowVersionValue);
+        }
+
+        if (registryUrl == null || bucketId == null || flowId == null) {
+            throw new IllegalArgumentException("Configuration does not provide the filename of the flow to run, a URL to fetch it from, or the registryUrl, bucketId, and flowId.");
+        }
+
+        try {
+            final SSLContext sslContext = SslConfigurationUtil.createSslContext(sslContextDefinition);
+            return fetchFlowFromRegistry(registryUrl, bucketId, flowId, flowVersion, sslContext);
+        } catch (final NiFiRegistryException e) {
+            throw new StatelessConfigurationException("Could not fetch flow from Registry", e);
+        }
+    }
+
+    private VersionedFlowSnapshot fetchFlowFromUrl(final String url, final SslContextDefinition sslContextDefinition) throws IOException {
+        final OkHttpClient.Builder clientBuilder = new OkHttpClient.Builder()
+            .callTimeout(30, TimeUnit.SECONDS);
+
+        if (sslContextDefinition != null) {
+            final TlsConfiguration tlsConfiguration = SslConfigurationUtil.createTlsConfiguration(sslContextDefinition);
+            OkHttpClientUtils.applyTlsToOkHttpClientBuilder(tlsConfiguration, clientBuilder);
+        }
+
+        final OkHttpClient client = clientBuilder.build();
+
+        final Request getRequest = new Request.Builder()
+            .url(url)
+            .get()
+            .build();
+
+        final Call call = client.newCall(getRequest);
+
+        try (final Response response = call.execute()) {
+            final ResponseBody responseBody = response.body();
+
+            if (!response.isSuccessful()) {
+                final String responseText = responseBody == null ? "<No Message Received from Server>" : responseBody.string();
+                throw new IOException("Failed to download flow from URL " + url + ": Response was " + response.code() + ": " + responseText);
+            }
+
+            if (responseBody == null) {
+                throw new IOException("Failed to download flow from URL " + url + ": Received successful response code " + response.code() + " but no Response body");
+            }
+
+            try {
+                final ObjectMapper objectMapper = new ObjectMapper();

Review comment:
       `ObjectMapper` is thread-safe and could be declared as an instance variable.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +

Review comment:
       Ah it didn't occur to me that that's an option - to create fewer task configs than `maxTasks`. Will do. Great catch!




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);

Review comment:
       Good catch. We should be calling `dataflow.purge()` if we're going to re-queue the data




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSinkTask.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.kafka.connect;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.sink.SinkRecord;
+import org.apache.kafka.connect.sink.SinkTask;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSinkTask extends SinkTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
+
+    private StatelessDataflow dataflow;
+    private String inputPortName;
+    private Set<String> failurePortNames;
+    private long timeoutMillis;
+    private Pattern headerNameRegex;
+    private String headerNamePrefix;
+    private int batchSize;
+    private long batchBytes;
+    private QueueSize queueSize;
+    private String dataflowName;
+
+    private long backoffMillis = 0L;
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+
+        final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
+        headerNameRegex = regex == null ? null : Pattern.compile(regex);
+        headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
+
+        batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
+        batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
+        final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
+        if (inputPortName == null) {
+            final Set<String> inputPorts = dataflow.getInputPortNames();
+            if (inputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+                    + "must have at least one Input Port at the root level.");
+            }
+
+            if (inputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+                    + "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
+            }
+
+            inputPortName = inputPorts.iterator().next();
+        }
+
+        // Validate the input port
+        if (!dataflow.getInputPortNames().contains(inputPortName)) {
+            throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+                + dataflow.getInputPortNames());
+        }
+
+        // Determine the failure Ports, if any are given.
+        final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
+        if (failurePortList == null || failurePortList.trim().isEmpty()) {
+            failurePortNames = Collections.emptySet();
+        } else {
+            failurePortNames = new HashSet<>();
+
+            final String[] names = failurePortList.split(",");
+            for (final String name : names) {
+                final String trimmed = name.trim();
+                failurePortNames.add(trimmed);
+            }
+        }
+
+        // Validate the failure ports
+        final Set<String> outputPortNames = dataflow.getOutputPortNames();
+        for (final String failurePortName : failurePortNames) {
+            if (!outputPortNames.contains(failurePortName)) {
+                throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+                    + " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
+            }
+        }
+    }
+
+    @Override
+    public void put(final Collection<SinkRecord> records) {
+        if (backoffMillis > 0) {
+            logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
+
+            try {
+                Thread.sleep(backoffMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
+            }
+        }
+
+        logger.debug("Enqueuing {} Kafka messages", records.size());
+
+        for (final SinkRecord record : records) {
+            final Map<String, String> attributes = createAttributes(record);
+            final byte[] contents = getContents(record.value());
+
+            queueSize = dataflow.enqueue(contents, attributes, inputPortName);
+        }
+
+        if (queueSize == null || queueSize.getObjectCount() < batchSize) {
+            return;
+        }
+        if (queueSize.getByteCount() < batchBytes) {
+            return;
+        }
+
+        logger.debug("Triggering dataflow");
+
+        try {
+            triggerDataflow();
+            resetBackoff();
+        } catch (final RetriableException re) {
+            backoff();
+            throw re;
+        }
+    }
+
+    private void backoff() {
+        // If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
+        if (backoffMillis == 0L) {
+            backoffMillis = 1000L;
+        }
+
+        backoffMillis = Math.min(backoffMillis * 2, 10_000L);
+    }
+
+    private void resetBackoff() {
+        backoffMillis = 0L;
+    }
+
+    private void triggerDataflow() {
+        final long start = System.nanoTime();
+        while (dataflow.isFlowFileQueued()) {
+            final DataflowTrigger trigger = dataflow.trigger();
+
+            try {
+                final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);

Review comment:
       Makes sense. Will address.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topic.name";
+
+    static final String TOPIC_NAME_ATTRIBUTE = "topic.name.attribute";
+    static final String KEY_ATTRIBUTE = "key.attribute";
+    static final String HEADER_REGEX = "header.attribute.regex";
+
+    private Map<String, String> properties;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {
+        this.properties = new HashMap<>(this.properties);
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return StatelessNiFiSourceTask.class;
+    }
+
+    @Override
+    public List<Map<String, String>> taskConfigs(final int maxTasks) {
+        final List<Map<String, String>> configs = new ArrayList<>();
+        for (int i=0; i < maxTasks; i++) {
+            configs.add(new HashMap<>(properties));

Review comment:
       In "NiFi proper" we have load-balanced connections that would allow the nodes to distribute the data between themselves to load-balance. We don't have that currently in Stateless NiFi. Not sure it would make sense to provide that in Stateless, given how it all works. I think for the moment, when interacting with a source that does not itself provide queuing/scaling semantics, we would just have to rely on the deployer making sure that the 'unit of work' that they deploy can be accomplished by a single task. There may be improvements that can be made to this in the future though.




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topics";
+
+    static final String TOPIC_NAME_ATTRIBUTE = "topic.name.attribute";
+    static final String KEY_ATTRIBUTE = "key.attribute";
+    static final String HEADER_REGEX = "header.attribute.regex";
+
+    private Map<String, String> properties;
+    private boolean primaryNodeOnly;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+
+        final StatelessDataflow dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+        dataflow.shutdown();
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {

Review comment:
       I wouldn't override reconfigure - the default implementation calls stop and start on the connector.
   If you do override it, you need to apply the new configurations - so everything done in start needs to be repeated here.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);

Review comment:
       @markap14, I misunderstood the offset commit flow of the source tasks. No need for this change, all of the records can have the same component state in their header. The framework will wait for all messages in the batch to be ack'd by kafka before committing the offsets.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port. Expected: " + expectedPortName + ". Actual: " + portName);
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents, final Map<String, ?> componentState, final Map<String, ?> partitionMap) {
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final ConnectHeaders headers = new ConnectHeaders();
+        if (headerAttributeNamePattern != null) {
+            // TODO: When we download/create the dataflow, create a hash of it. Then save that state. When we do it next time,
+            //       compare the hash to the last one. If changed, need to trigger connect framework to tell it that the config has changed.
+            //       Would be done via Source/Sink Context.
+            //       Or perhaps we should include the flow JSON itself in the configuration... would require that we string-ify the JSON though. This would be the cleanest, though. Would be optional.
+            //       We can just document that you either include it inline, or you don't make changes to the dataflow; instead, save as a separate dataflow and update task to point to the new one.
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+                    headers.add(headerName, headerValue, Schema.STRING_SCHEMA);
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partitionMap, componentState, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {
+        super.commitRecord(record, metadata);
+
+        final long unacked = unacknowledgedRecords.decrementAndGet();
+        logger.debug("SourceRecord {} committed; number of unacknowledged FlowFiles is now {}", record, unacked);
+
+        if (unacked < 1) {

Review comment:
       Thanks for the explanation, makes sense




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.nifi.kafka.connect.validators.ConnectDirectoryExistsValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectFileExistsOrUrlValidator;
+import org.apache.nifi.kafka.connect.validators.ConnectHttpUrlValidator;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterOverride;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StatelessKafkaConnectorUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessKafkaConnectorUtil.class);
+    private static final Lock unpackNarLock = new ReentrantLock();
+
+    static final String NAR_DIRECTORY = "nar.directory";
+    static final String WORKING_DIRECTORY = "working.directory";
+    static final String FLOW_SNAPSHOT = "flow.snapshot";
+    static final String KRB5_FILE = "krb5.file";
+    static final String NEXUS_BASE_URL = "nexus.url";
+    static final String DATAFLOW_TIMEOUT = "dataflow.timeout";
+
+    static final String TRUSTSTORE_FILE = "security.truststore";
+    static final String TRUSTSTORE_TYPE = "security.truststoreType";
+    static final String TRUSTSTORE_PASSWORD = "security.truststorePasswd";
+    static final String KEYSTORE_FILE = "security.keystore";
+    static final String KEYSTORE_TYPE = "security.keystoreType";
+    static final String KEYSTORE_PASSWORD = "security.keystorePasswd";
+    static final String KEY_PASSWORD = "security.keyPasswd";
+
+    static final String DEFAULT_KRB5_FILE = "/etc/krb5.conf";
+    static final String DEFAULT_DATAFLOW_TIMEOUT = "60 sec";
+    static final File DEFAULT_WORKING_DIRECTORY = new File("/tmp/nifi-stateless-working");
+
+    private static final Pattern STATELESS_BOOTSTRAP_FILE_PATTERN = Pattern.compile("nifi-stateless-bootstrap-(.*).jar");
+    private static final Pattern PARAMETER_WITH_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*?):(.*)");
+    private static final Pattern PARAMETER_WITHOUT_CONTEXT_PATTERN = Pattern.compile("parameter\\.(.*)");
+
+    public static void addCommonConfigElements(final ConfigDef configDef) {
+        configDef.define(NAR_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the directory that stores the NiFi Archives (NARs)");
+        configDef.define(WORKING_DIRECTORY, ConfigDef.Type.STRING, null, new ConnectDirectoryExistsValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the temporary working directory for expanding NiFi Archives (NARs)");
+        configDef.define(FLOW_SNAPSHOT, ConfigDef.Type.STRING, null, new ConnectFileExistsOrUrlValidator(), ConfigDef.Importance.HIGH,
+            "Specifies the file containing the dataflow to run");
+
+        configDef.define(StatelessKafkaConnectorUtil.KRB5_FILE, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_KRB5_FILE, ConfigDef.Importance.MEDIUM,
+            "Specifies the krb5.conf file to use if connecting to Kerberos-enabled services");
+        configDef.define(StatelessKafkaConnectorUtil.NEXUS_BASE_URL, ConfigDef.Type.STRING, null, new ConnectHttpUrlValidator(), ConfigDef.Importance.MEDIUM,
+            "Specifies the Base URL of the Nexus instance to source extensions from");
+
+        configDef.define(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, ConfigDef.Type.STRING, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT, ConfigDef.Importance.MEDIUM,
+            "Specifies the amount of time to wait for the dataflow to finish processing input before considering the dataflow a failure");
+
+        configDef.define(KEYSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the keystore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications.");
+        configDef.define(KEYSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Keystore file. Either JKS or PKCS12.");
+        configDef.define(KEYSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the keystore.");
+        configDef.define(KEY_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the key in the keystore. If not provided, the password is assumed to be the same as the keystore password.");
+        configDef.define(TRUSTSTORE_FILE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "Filename of the truststore that Stateless NiFi should use for connecting to NiFi Registry and for Site-to-Site communications. If not specified, communications will occur only over " +
+                "http, not https.");
+        configDef.define(TRUSTSTORE_TYPE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM,
+            "The type of the Truststore file. Either JKS or PKCS12.");
+        configDef.define(TRUSTSTORE_PASSWORD, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.MEDIUM,
+            "The password for the truststore.");
+    }
+
+    public static String getVersion() {
+        final File bootstrapJar = detectBootstrapJar();
+        if (bootstrapJar == null) {
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        try (final JarFile jarFile = new JarFile(bootstrapJar)) {
+            final Manifest manifest = jarFile.getManifest();
+            if (manifest != null) {
+                return manifest.getMainAttributes().getValue("Implementation-Version");
+            }
+        } catch (IOException e) {
+            logger.warn("Could not determine Version of NiFi Stateless Kafka Connector", e);
+            return "<Unable to Stateless NiFi Kafka Connector Version>";
+        }
+
+        return "<Unable to Stateless NiFi Kafka Connector Version>";
+    }
+
+    public static StatelessDataflow createDataflow(final Map<String, String> properties) {
+        final StatelessEngineConfiguration engineConfiguration = createEngineConfiguration(properties);
+        final String configuredFlowSnapshot = properties.get(FLOW_SNAPSHOT);
+
+        final List<ParameterOverride> parameterOverrides = parseParameterOverrides(properties);
+        final String dataflowName = properties.get("name");
+
+        final DataflowDefinition<?> dataflowDefinition;
+        final StatelessBootstrap bootstrap;
+        try {
+            final Map<String, String> dataflowDefinitionProperties = new HashMap<>();
+
+            if (configuredFlowSnapshot.startsWith("http://") || configuredFlowSnapshot.startsWith("https://")) {
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.url", configuredFlowSnapshot);
+            } else {
+                final File flowSnapshotFile = new File(configuredFlowSnapshot);
+                dataflowDefinitionProperties.put("nifi.stateless.flow.snapshot.file", flowSnapshotFile.getAbsolutePath());
+            }
+
+            dataflowDefinitionProperties.put("nifi.stateless.flow.name", dataflowName);
+
+            MDC.setContextMap(Collections.singletonMap("dataflow", dataflowName));
+
+            // Use a Write Lock to ensure that only a single thread is calling StatelessBootstrap.bootstrap().
+            // We do this because the bootstrap() method will expand all NAR files into the working directory.
+            // If we have multiple Connector instances, or multiple tasks, we don't want several threads all
+            // unpacking NARs at the same time, as it could potentially result in the working directory becoming corrupted.
+            unpackNarLock.lock();
+            try {
+                bootstrap = StatelessBootstrap.bootstrap(engineConfiguration, StatelessNiFiSourceTask.class.getClassLoader());
+            } finally {
+                unpackNarLock.unlock();
+            }
+
+            dataflowDefinition = bootstrap.parseDataflowDefinition(dataflowDefinitionProperties);
+            return bootstrap.createDataflow(dataflowDefinition, parameterOverrides);
+        } catch (final Exception e) {
+            throw new RuntimeException("Failed to bootstrap Stateless NiFi Engine", e);
+        }
+    }
+
+    private static List<ParameterOverride> parseParameterOverrides(final Map<String, String> properties) {
+        final List<ParameterOverride> parameterOverrides = new ArrayList<>();
+
+        for (final Map.Entry<String, String> entry : properties.entrySet()) {
+            final String parameterValue = entry.getValue();
+
+            ParameterOverride parameterOverride = null;
+            final Matcher matcher = PARAMETER_WITH_CONTEXT_PATTERN.matcher(entry.getKey());
+            if (matcher.matches()) {
+                final String contextName = matcher.group(1);
+                final String parameterName = matcher.group(2);
+                parameterOverride = new ParameterOverride(contextName, parameterName, parameterValue);
+            } else {
+                final Matcher noContextMatcher = PARAMETER_WITHOUT_CONTEXT_PATTERN.matcher(entry.getKey());
+                if (noContextMatcher.matches()) {
+                    final String parameterName = noContextMatcher.group(1);
+                    parameterOverride = new ParameterOverride(parameterName, parameterValue);
+                }
+            }
+
+            if (parameterOverride != null) {
+                parameterOverrides.add(parameterOverride);
+            }
+        }
+
+        return parameterOverrides;
+    }
+
+    private static StatelessEngineConfiguration createEngineConfiguration(final Map<String, String> properties) {
+        final File narDirectory;
+        final String narDirectoryFilename = properties.get(NAR_DIRECTORY);
+        if (narDirectoryFilename == null) {
+            narDirectory = detectNarDirectory();
+        } else {
+            narDirectory = new File(narDirectoryFilename);
+        }
+
+        final File workingDirectory;
+        final String workingDirectoryFilename = properties.get(WORKING_DIRECTORY);
+        if (workingDirectoryFilename == null) {
+            workingDirectory = DEFAULT_WORKING_DIRECTORY;
+        } else {
+            workingDirectory = new File(workingDirectoryFilename);
+        }
+
+        final SslContextDefinition sslContextDefinition = createSslContextDefinition(properties);
+
+        final StatelessEngineConfiguration engineConfiguration = new StatelessEngineConfiguration() {
+            @Override
+            public File getWorkingDirectory() {
+                return workingDirectory;
+            }
+
+            @Override
+            public File getNarDirectory() {
+                return narDirectory;
+            }
+
+            @Override
+            public File getKrb5File() {
+                return new File(properties.getOrDefault(KRB5_FILE, DEFAULT_KRB5_FILE));
+            }
+
+            @Override
+            public SslContextDefinition getSslContext() {
+                return sslContextDefinition;
+            }
+
+            @Override
+            public String getSensitivePropsKey() {
+                return "nifi-stateless";

Review comment:
       Yeah, I forgot to come back to that. 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.

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



[GitHub] [nifi] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topics";
+
+    static final String TOPIC_NAME_ATTRIBUTE = "topic.name.attribute";
+    static final String KEY_ATTRIBUTE = "key.attribute";
+    static final String HEADER_REGEX = "header.attribute.regex";
+
+    private Map<String, String> properties;
+    private boolean primaryNodeOnly;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+
+        final StatelessDataflow dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+        dataflow.shutdown();
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {

Review comment:
       I wouldn't override reconfigure - the default implementation calls stop and start on the connector.
   If you do override it, you need to apply the new configurations - so everything done in start needs to be repeated here.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);

Review comment:
       @markap14, I misunderstood the offset commit flow of the source tasks. No need for this change, all of the records can have the same component state in their header. The framework will wait for all messages in the batch to be ack'd by kafka before committing the offsets.

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port. Expected: " + expectedPortName + ". Actual: " + portName);
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents, final Map<String, ?> componentState, final Map<String, ?> partitionMap) {
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final ConnectHeaders headers = new ConnectHeaders();
+        if (headerAttributeNamePattern != null) {
+            // TODO: When we download/create the dataflow, create a hash of it. Then save that state. When we do it next time,
+            //       compare the hash to the last one. If changed, need to trigger connect framework to tell it that the config has changed.
+            //       Would be done via Source/Sink Context.
+            //       Or perhaps we should include the flow JSON itself in the configuration... would require that we string-ify the JSON though. This would be the cleanest, though. Would be optional.
+            //       We can just document that you either include it inline, or you don't make changes to the dataflow; instead, save as a separate dataflow and update task to point to the new one.
+
+            for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
+                if (headerAttributeNamePattern.matcher(entry.getKey()).matches()) {
+                    final String headerName = entry.getKey();
+                    final String headerValue = entry.getValue();
+                    headers.add(headerName, headerValue, Schema.STRING_SCHEMA);
+                }
+            }
+        }
+
+        final Object key = keyAttributeName == null ? null : flowFile.getAttribute(keyAttributeName);
+        final Schema keySchema = key == null ? null : Schema.STRING_SCHEMA;
+        final Long timestamp = System.currentTimeMillis();
+
+        return new SourceRecord(partitionMap, componentState, topic, topicPartition, keySchema, key, valueSchema, contents, timestamp, headers);
+    }
+
+    @Override
+    public void commitRecord(final SourceRecord record, final RecordMetadata metadata) throws InterruptedException {
+        super.commitRecord(record, metadata);
+
+        final long unacked = unacknowledgedRecords.decrementAndGet();
+        logger.debug("SourceRecord {} committed; number of unacknowledged FlowFiles is now {}", record, unacked);
+
+        if (unacked < 1) {

Review comment:
       Thanks for the explanation, makes sense

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       So the source topic partition is identified by the task index, is that correct?
   My question is, what happens if we start a connector with 3 tasks, it starts working and committing offsets with task ids 0, 1, 2. Then, we reconfigure the connector, and reduce the number of tasks to 2. Now we have task ids 0, 1. What happens with the committed offsets of task no. 2? Can the remaining 2 tasks pick up the work which was originally done by task no. 2?

##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+
+    private Map<String, ?> previousBatchComponentStates = Collections.emptyMap();
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);

Review comment:
       Thanks, I see - then I don't really see the added value of the local state, and seems like a good way of shooting yourself in the leg when configuring the connector




----------------------------------------------------------------
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] urbandan commented on a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceConnector.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.connect.connector.Task;
+import org.apache.kafka.connect.source.SourceConnector;
+import org.apache.nifi.kafka.connect.validators.ConnectRegularExpressionValidator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class StatelessNiFiSourceConnector extends SourceConnector {
+    static final String OUTPUT_PORT_NAME = "output.port";
+    static final String TOPIC_NAME = "topic.name";
+
+    static final String TOPIC_NAME_ATTRIBUTE = "topic.name.attribute";
+    static final String KEY_ATTRIBUTE = "key.attribute";
+    static final String HEADER_REGEX = "header.attribute.regex";
+
+    private Map<String, String> properties;
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        this.properties = new HashMap<>(properties);
+    }
+
+    @Override
+    public void reconfigure(final Map<String, String> properties) {
+        this.properties = new HashMap<>(this.properties);
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return StatelessNiFiSourceTask.class;
+    }
+
+    @Override
+    public List<Map<String, String>> taskConfigs(final int maxTasks) {
+        final List<Map<String, String>> configs = new ArrayList<>();
+        for (int i=0; i < maxTasks; i++) {
+            configs.add(new HashMap<>(properties));

Review comment:
       At this phase, the connector has to have some information about the input data, and partition that between the created tasks. Not sure how hard it is, or if the stateless nifi flows can coordinate under the hood.




----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.Header;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents);
+            sourceRecords.add(sourceRecord);
+        }
+
+        logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
+
+        // If there is at least one record, we don't want to acknowledge the trigger result until Kafka has committed the Record.
+        // This is handled by incrementing the unacknkowledgedRecords count. Then, Kafka Connect will call this.commitRecords().
+        // The commitRecords() call will then decrement the number of unacknowledgedRecords, and when all unacknowledged Records have been
+        // acknowledged, it will acknowledge the trigger result.
+        //
+        // However, if there are no records, this.commitRecords() will never be called. As a result, we need toe nsure that we acknowledge the trigger result here.
+        if (sourceRecords.size() > 0) {
+            unacknowledgedRecords.addAndGet(sourceRecords.size());
+        } else {
+            triggerResult.acknowledge();
+        }
+
+        return sourceRecords;
+    }
+
+    private void verifyFlowFilesTransferredToProperPort(final TriggerResult triggerResult, final String expectedPortName, final DataflowTrigger trigger) {
+        final Map<String, List<FlowFile>> flowFileOutputMap = triggerResult.getOutputFlowFiles();
+
+        for (final Map.Entry<String, List<FlowFile>> entry : flowFileOutputMap.entrySet()) {
+            final String portName = entry.getKey();
+            final List<FlowFile> flowFiles = entry.getValue();
+
+            if (!flowFiles.isEmpty() && !expectedPortName.equals(portName)) {
+                logger.error("Dataflow transferred FlowFiles to Port {} but was expecting data to be transferred to {}. Rolling back session.", portName, expectedPortName);
+                trigger.cancel();
+                throw new RetriableException("Data was transferred to unexpected port");
+            }
+        }
+    }
+
+
+    private SourceRecord createSourceRecord(final FlowFile flowFile, final byte[] contents) {
+        final Map<String, ?> partition = Collections.emptyMap();
+        final Map<String, ?> sourceOffset = Collections.emptyMap();
+        final Schema valueSchema = (contents == null || contents.length == 0) ? null : Schema.BYTES_SCHEMA;
+
+        // Kafka Connect currently gives us no way to determine the number of partitions that a given topic has.
+        // Therefore, we have no way to partition based on an attribute or anything like that, unless we left it up to
+        // the dataflow developer to know how many partitions exist a priori and explicitly set an attribute in the range of 0..max,
+        // but that is not a great solution. Kafka does support using a Simple Message Transform to change the partition of a given
+        // record, so that may be the best solution.
+        final Integer topicPartition = null;
+
+        final String topic;
+        if (topicNameAttribute == null) {
+            topic = topicName;
+        } else {
+            final String attributeValue = flowFile.getAttribute(topicNameAttribute);
+            topic = attributeValue == null ? topicName : attributeValue;
+        }
+
+        final List<Header> headers;

Review comment:
       I looked for this but somehow i couldn't find the ConnectHeader class in my IDE. Was able to find it now, since I know what to look for. Will use this. 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.

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



[GitHub] [nifi] markap14 commented on pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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


   @urbandan @heritamas let me know if there's anything else that you noticed that I missed. Otherwise, I think we can probably get this merged in shortly.


----------------------------------------------------------------
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 a change in pull request #4730: NIFI-8095: Created StatelessNiFi Sink Connector and Source Connector.…

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



##########
File path: nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessNiFiSourceTask.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.kafka.connect;
+
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.errors.RetriableException;
+import org.apache.kafka.connect.header.ConnectHeaders;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.source.SourceTask;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+public class StatelessNiFiSourceTask extends SourceTask {
+    public static final String STATE_MAP_KEY = "task.index";
+    private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
+
+    private StatelessDataflow dataflow;
+    private String outputPortName;
+    private String topicName;
+    private String topicNameAttribute;
+    private TriggerResult triggerResult;
+    private String keyAttributeName;
+    private Pattern headerAttributeNamePattern;
+    private long timeoutMillis;
+    private String dataflowName;
+    private long failureYieldExpiration = 0L;
+
+    private final Map<String, String> clusterStatePartitionMap = Collections.singletonMap(STATE_MAP_KEY, "CLUSTER");
+    private Map<String, String> localStatePartitionMap = new HashMap<>();
+    private boolean primaryNodeOnly;
+    private boolean primaryNodeTask;
+
+    private final AtomicLong unacknowledgedRecords = new AtomicLong(0L);
+
+    @Override
+    public String version() {
+        return StatelessKafkaConnectorUtil.getVersion();
+    }
+
+    @Override
+    public void start(final Map<String, String> properties) {
+        logger.info("Starting Source Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
+
+        final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
+        timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
+
+        topicName = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME);
+        topicNameAttribute = properties.get(StatelessNiFiSourceConnector.TOPIC_NAME_ATTRIBUTE);
+        keyAttributeName = properties.get(StatelessNiFiSourceConnector.KEY_ATTRIBUTE);
+
+        if (topicName == null && topicNameAttribute == null) {
+            throw new ConfigException("Either the topic.name or topic.name.attribute configuration must be specified");
+        }
+
+        final String headerRegex = properties.get(StatelessNiFiSourceConnector.HEADER_REGEX);
+        headerAttributeNamePattern = headerRegex == null ? null : Pattern.compile(headerRegex);
+
+        dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
+        primaryNodeOnly = dataflow.isSourcePrimaryNodeOnly();
+
+        // Determine the name of the Output Port to retrieve data from
+        dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
+        outputPortName = properties.get(StatelessNiFiSourceConnector.OUTPUT_PORT_NAME);
+        if (outputPortName == null) {
+            final Set<String> outputPorts = dataflow.getOutputPortNames();
+            if (outputPorts.isEmpty()) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Output Port at the root level. Dataflows used for a Kafka Connect Source Task "
+                    + "must have at least one Output Port at the root level.");
+            }
+
+            if (outputPorts.size() > 1) {
+                throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Output Ports at the root level (" + outputPorts.toString()
+                    + "). The " + StatelessNiFiSourceConnector.OUTPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be retrieved from.");
+            }
+
+            outputPortName = outputPorts.iterator().next();
+        }
+
+        final String taskIndex = properties.get(STATE_MAP_KEY);
+        localStatePartitionMap.put(STATE_MAP_KEY, taskIndex);
+        primaryNodeTask = "0".equals(taskIndex);
+
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.warn("Configured Dataflow ({}) requires that the source be run only on the Primary Node, but the Connector is configured with more than one task. The dataflow will only be run by" +
+                " one of the tasks.", dataflowName);
+        }
+
+        final Map<String, String> localStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(localStatePartitionMap);
+        final Map<String, String> clusterStateMap = (Map<String, String>) (Map) context.offsetStorageReader().offset(clusterStatePartitionMap);
+
+        dataflow.setComponentStates(localStateMap, Scope.LOCAL);
+        dataflow.setComponentStates(clusterStateMap, Scope.CLUSTER);
+    }
+
+    @Override
+    public List<SourceRecord> poll() throws InterruptedException {
+        final long yieldExpiration = Math.max(failureYieldExpiration, dataflow.getSourceYieldExpiration());
+        final long now = System.currentTimeMillis();
+        final long yieldMillis = yieldExpiration - now;
+        if (yieldMillis > 0) {
+            // If source component has yielded, we don't want to trigger it again until the yield expiration expires, in order to avoid
+            // overloading the source system.
+            logger.debug("Source of NiFi flow has opted to yield for {} milliseconds. Will pause dataflow until that time period has elapsed.", yieldMillis);
+            Thread.sleep(yieldMillis);
+            return null;
+        }
+
+        // If the source of the dataflow requires that the task be run on Primary Node Only, and this is not Task 0, then
+        // we do not want to run the task.
+        if (primaryNodeOnly && !primaryNodeTask) {
+            logger.debug("Source of dataflow {} is to be run on Primary Node only, and this task is not the Primary Node task. Will not trigger dataflow.", dataflow);
+            return null;
+        }
+
+        if (unacknowledgedRecords.get() > 0) {
+            // If we have records that haven't yet been acknowledged, we want to return null instead of running.
+            // We need to wait for the last results to complete before triggering the dataflow again.
+            return null;
+        }
+
+        logger.debug("Triggering dataflow");
+        final long start = System.nanoTime();
+
+        final DataflowTrigger trigger = dataflow.trigger();
+        final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        if (!resultOptional.isPresent()) {
+            logger.warn("Dataflow timed out after waiting {} milliseconds. Will cancel the execution.", timeoutMillis);
+            trigger.cancel();
+            return null;
+        }
+
+        triggerResult = resultOptional.get();
+
+        if (!triggerResult.isSuccessful()) {
+            logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
+            trigger.cancel();
+            failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
+            return null;
+        }
+
+        // Verify that data was only transferred to the expected Output Port
+        verifyFlowFilesTransferredToProperPort(triggerResult, outputPortName, trigger);
+
+        final long nanos = System.nanoTime() - start;
+
+        final List<FlowFile> outputFlowFiles = triggerResult.getOutputFlowFiles(outputPortName);
+        final List<SourceRecord> sourceRecords = new ArrayList<>(outputFlowFiles.size());
+
+        Map<String, ?> componentState = dataflow.getComponentStates(Scope.CLUSTER);
+        final Map<String, ?> partitionMap;
+        if (componentState == null || componentState.isEmpty()) {
+            componentState = dataflow.getComponentStates(Scope.LOCAL);
+            partitionMap = localStatePartitionMap;
+        } else {
+            partitionMap = clusterStatePartitionMap;
+        }
+
+        for (final FlowFile flowFile : outputFlowFiles) {
+            final byte[] contents = triggerResult.readContent(flowFile);
+            final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);

Review comment:
       Yes, will do.




----------------------------------------------------------------
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