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:
[email protected]


Reply via email to