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