exceptionfactory commented on code in PR #8463:
URL: https://github.com/apache/nifi/pull/8463#discussion_r1526461916


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/README.md:
##########
@@ -0,0 +1,14 @@
+<!--
+  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.
+-->
+
+### Kafka3ProducerService
+- As the `send()` API maps cleanly to the context of a single `FlowFile`, 
additional APIs are useful to handle the ability to publish multiple FlowFiles 
in the context of a single publish to Kafka.

Review Comment:
   Is there a reason for this separate readme as opposed to including these 
comments in the class itself?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-integration/src/test/java/org/apache/nifi/kafka/processors/publish/additional/readme.txt:
##########
@@ -0,0 +1,2 @@
+Integration tests derived from examples provided in:
+https://github.com/apache/nifi/blob/main/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6/additionalDetails.html

Review Comment:
   Is this file necessary? Recommend changing to Markdown for consistency if 
you prefer to keep it.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-integration/src/test/java/org/apache/nifi/kafka/processors/PublishKafkaWrapperRecordIT.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.processors;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.commons.io.IOUtils;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.header.Header;
+import org.apache.nifi.kafka.shared.property.PublishStrategy;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.MethodOrderer;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestMethodOrder;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@TestMethodOrder(MethodOrderer.MethodName.class)
+public class PublishKafkaWrapperRecordIT extends AbstractPublishKafkaIT {
+    private static final String TEST_RESOURCE = 
"org/apache/nifi/kafka/processors/publish/ffwrapper.json";
+
+    private static final String KEY_ATTRIBUTE_KEY = "keyAttribute";
+    private static final String KEY_ATTRIBUTE_VALUE = "keyAttributeValue";
+
+    private static final int TEST_RECORD_COUNT = 3;
+
+    @Test
+    public void test_1_KafkaTestContainerProduceOneFlowFile() throws 
InitializationException, IOException {
+        final TestRunner runner = 
TestRunners.newTestRunner(PublishKafka.class);
+        runner.setValidateExpressionUsage(false);
+        runner.setProperty(PublishKafka.CONNECTION_SERVICE, 
addKafkaConnectionService(runner));
+        addRecordReaderService(runner);
+        addRecordWriterService(runner);
+        addRecordKeyWriterService(runner);
+
+        runner.setProperty(PublishKafka.TOPIC_NAME, getClass().getName());
+        runner.setProperty(PublishKafka.KEY, KEY_ATTRIBUTE_KEY);
+        runner.setProperty(PublishKafka.MESSAGE_KEY_FIELD, "address");
+        runner.setProperty(PublishKafka.ATTRIBUTE_NAME_REGEX, "a.*");
+        runner.setProperty(PublishKafka.PUBLISH_STRATEGY, 
PublishStrategy.USE_WRAPPER.name());
+
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(KEY_ATTRIBUTE_KEY, KEY_ATTRIBUTE_VALUE);
+        attributes.put("a1", "valueA1");
+        attributes.put("b1", "valueB1");
+        final byte[] bytesFlowFile = 
IOUtils.toByteArray(Objects.requireNonNull(
+                getClass().getClassLoader().getResource(TEST_RESOURCE)));
+        runner.enqueue(bytesFlowFile, attributes);
+        runner.run(1);
+        runner.assertAllFlowFilesTransferred(PublishKafka.REL_SUCCESS, 1);
+    }
+
+    @Test
+    public void test_2_KafkaTestContainerConsumeMultipleRecords() throws 
IOException {
+        try (KafkaConsumer<String, String> consumer = new 
KafkaConsumer<>(getKafkaConsumerProperties())) {
+            
consumer.subscribe(Collections.singletonList(getClass().getName()));
+            final ConsumerRecords<String, String> records = 
consumer.poll(DURATION_POLL);
+            assertEquals(TEST_RECORD_COUNT, records.count());
+            for (ConsumerRecord<String, String> record : records) {
+                // kafka record headers
+                final List<Header> headers = 
Arrays.asList(record.headers().toArray());
+                assertEquals(1, headers.size());
+                final Header header = record.headers().iterator().next();
+                assertEquals("a1", header.key());
+                assertEquals("valueA1", new String(header.value(), 
StandardCharsets.UTF_8));
+                // kafka record key
+                final ObjectNode kafkaKey = (ObjectNode) 
objectMapper.readTree(record.key());
+                assertNotNull(kafkaKey);
+                assertEquals("Main", kafkaKey.get("street-name").textValue());
+                assertEquals(5, (kafkaKey.get("street-number").intValue() % 
100));
+                // kafka record value (wrapped record)
+                final ObjectNode kafkaValue = (ObjectNode) 
objectMapper.readTree(record.value());
+                assertNotNull(kafkaValue);
+                assertNotEquals(0, kafkaValue.get("id").asInt());
+                assertEquals(1, kafkaValue.get("name").asText().length());
+                assertTrue(kafkaValue.get("address") instanceof ObjectNode);

Review Comment:
   The `assertInstanceOf()` method can be used instead of this approach.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/producer/Kafka3ProducerService.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.service.producer;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.common.ServiceConfiguration;
+import org.apache.nifi.kafka.service.api.producer.FlowFileResult;
+import org.apache.nifi.kafka.service.api.producer.KafkaProducerService;
+import org.apache.nifi.kafka.service.api.producer.ProducerConfiguration;
+import org.apache.nifi.kafka.service.api.producer.PublishContext;
+import org.apache.nifi.kafka.service.api.producer.RecordSummary;
+import org.apache.nifi.kafka.service.api.record.KafkaRecord;
+import 
org.apache.nifi.kafka.service.producer.txn.KafkaNonTransactionalProducerWrapper;
+import org.apache.nifi.kafka.service.producer.txn.KafkaProducerWrapper;
+import 
org.apache.nifi.kafka.service.producer.txn.KafkaTransactionalProducerWrapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+public class Kafka3ProducerService implements KafkaProducerService {
+    private final Logger logger = LoggerFactory.getLogger(getClass());
+
+    private final Producer<byte[], byte[]> producer;
+    private final List<ProducerCallback> callbacks;
+
+    private final ServiceConfiguration serviceConfiguration;
+
+    private final KafkaProducerWrapper wrapper;
+
+    public Kafka3ProducerService(final Properties properties,
+                                 final ServiceConfiguration 
serviceConfiguration,
+                                 final ProducerConfiguration 
producerConfiguration) {
+        final ByteArraySerializer serializer = new ByteArraySerializer();
+        this.producer = new KafkaProducer<>(properties, serializer, 
serializer);
+        this.callbacks = new ArrayList<>();
+
+        this.serviceConfiguration = serviceConfiguration;
+
+        this.wrapper = producerConfiguration.getUseTransactions()
+                ? new KafkaTransactionalProducerWrapper(producer)
+                : new KafkaNonTransactionalProducerWrapper(producer);
+    }
+
+    @Override
+    public void close() {
+        producer.close();
+    }
+
+    @Override
+    public void init() {
+        wrapper.init();
+        logger.trace("init()");

Review Comment:
   Are these trace logs necessary, or is similar information available from the 
Kafka client logging?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/pom.xml:
##########
@@ -0,0 +1,99 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
https://maven.apache.org/xsd/maven-4.0.0.xsd";>
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-kafka-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-kafka-3-service</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-service-api</artifactId>
+            <version>2.0.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <version>2.0.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-user-service-api</artifactId>
+            <version>2.0.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>2.0.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kafka-shared</artifactId>
+            <version>2.0.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-pool2</artifactId>
+            <version>2.11.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>kafka</artifactId>
+            <version>${testcontainers.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <version>3.2.3</version>

Review Comment:
   The latest Kafka version is now `3.7.0` so updating this version would be 
helpful to start things off with the current version.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/producer/txn/KafkaProducerWrapper.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.service.producer.txn;
+
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.nifi.kafka.service.api.producer.PublishContext;
+import org.apache.nifi.kafka.service.api.record.KafkaRecord;
+import org.apache.nifi.kafka.service.producer.ProducerCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Abstract away the configured transactionality of the PublishKafka producer.
+ */
+public abstract class KafkaProducerWrapper {
+    protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+    protected final Producer<byte[], byte[]> producer;
+
+    public KafkaProducerWrapper(final Producer<byte[], byte[]> producer) {
+        this.producer = producer;
+    }
+
+    /**
+     * Transaction-enabled publish to Kafka involves the use of special Kafka 
client library APIs.
+     */
+    public abstract void init();
+
+    public void send(final Iterator<KafkaRecord> kafkaRecords, final 
PublishContext publishContext, final ProducerCallback callback) {
+        while (kafkaRecords.hasNext()) {
+            final KafkaRecord kafkaRecord = kafkaRecords.next();
+            producer.send(toProducerRecord(kafkaRecord, publishContext), 
callback);
+            callback.send();
+        }
+        logger.trace("send():inFlight");

Review Comment:
   Is this necessary?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.processors;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.processors.common.KafkaUtils;
+import org.apache.nifi.kafka.processors.consumer.OffsetTracker;
+import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
+import org.apache.nifi.kafka.processors.consumer.bundle.ByteRecordBundler;
+import 
org.apache.nifi.kafka.processors.consumer.convert.FlowFileStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.processors.consumer.convert.KafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.RecordStreamKafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.consumer.PollingContext;
+import org.apache.nifi.kafka.service.api.record.ByteRecord;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.KeyFormat;
+import org.apache.nifi.kafka.shared.property.OutputStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.StringUtils;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"kafka", "consumer", "record"})
+public class ConsumeKafka extends AbstractProcessor implements 
VerifiableProcessor {
+
+    static final AllowableValue TOPIC_NAME = new AllowableValue("names", 
"names", "Topic is a full topic name or comma separated list of names");
+    static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", 
"pattern", "Topic is a regex using the Java Pattern syntax");
+
+    static final PropertyDescriptor CONNECTION_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("Kafka Connection Service")
+            .displayName("Kafka Connection Service")
+            .description("Provides connections to Kafka Broker for publishing 
Kafka Records")
+            .identifiesControllerService(KafkaConnectionService.class)
+            .expressionLanguageSupported(NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
+            .name("Group ID")
+            .displayName("Group ID")
+            .description("Kafka Consumer Group Identifier corresponding to 
Kafka group.id property")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(NONE)
+            .build();
+
+    static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
+            .name("topic")
+            .displayName("Topic Name(s)")
+            .description("The name of the Kafka Topic(s) from which the 
Processor consumes Kafka Records. More than one can be supplied if comma 
separated.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .build();
+
+    static final PropertyDescriptor TOPIC_TYPE = new 
PropertyDescriptor.Builder()
+            .name("topic_type")
+            .displayName("Topic Name Format")

Review Comment:
   ```suggestion
               .name("Topic Name Format")
               .displayName("Topic Name Format")
   ```



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.processors;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.processors.common.KafkaUtils;
+import org.apache.nifi.kafka.processors.consumer.OffsetTracker;
+import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
+import org.apache.nifi.kafka.processors.consumer.bundle.ByteRecordBundler;
+import 
org.apache.nifi.kafka.processors.consumer.convert.FlowFileStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.processors.consumer.convert.KafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.RecordStreamKafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.consumer.PollingContext;
+import org.apache.nifi.kafka.service.api.record.ByteRecord;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.KeyFormat;
+import org.apache.nifi.kafka.shared.property.OutputStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.StringUtils;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"kafka", "consumer", "record"})
+public class ConsumeKafka extends AbstractProcessor implements 
VerifiableProcessor {
+
+    static final AllowableValue TOPIC_NAME = new AllowableValue("names", 
"names", "Topic is a full topic name or comma separated list of names");
+    static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", 
"pattern", "Topic is a regex using the Java Pattern syntax");
+
+    static final PropertyDescriptor CONNECTION_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("Kafka Connection Service")
+            .displayName("Kafka Connection Service")
+            .description("Provides connections to Kafka Broker for publishing 
Kafka Records")
+            .identifiesControllerService(KafkaConnectionService.class)
+            .expressionLanguageSupported(NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
+            .name("Group ID")
+            .displayName("Group ID")
+            .description("Kafka Consumer Group Identifier corresponding to 
Kafka group.id property")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(NONE)
+            .build();
+
+    static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
+            .name("topic")
+            .displayName("Topic Name(s)")
+            .description("The name of the Kafka Topic(s) from which the 
Processor consumes Kafka Records. More than one can be supplied if comma 
separated.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .build();
+
+    static final PropertyDescriptor TOPIC_TYPE = new 
PropertyDescriptor.Builder()
+            .name("topic_type")
+            .displayName("Topic Name Format")
+            .description("Specifies whether the Topic(s) provided are a comma 
separated list of names or a single regular expression")

Review Comment:
   ```suggestion
               .description("Specifies whether the Topics provided are a comma 
separated list of names or a single regular expression")
   ```



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.service;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.security.plain.PlainLoginModule;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.VerifiableControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.ServiceConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.producer.KafkaProducerService;
+import org.apache.nifi.kafka.service.api.producer.ProducerConfiguration;
+import org.apache.nifi.kafka.service.consumer.Kafka3ConsumerService;
+import org.apache.nifi.kafka.service.producer.Kafka3ProducerService;
+import org.apache.nifi.kafka.shared.property.SaslMechanism;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import 
org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_LOCATION;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_TYPE;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEY_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_LOCATION;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_TYPE;
+
+public class Kafka3ConnectionService extends AbstractControllerService 
implements KafkaConnectionService, VerifiableControllerService {
+
+    public static final PropertyDescriptor BOOTSTRAP_SERVERS = new 
PropertyDescriptor.Builder()
+            .name("bootstrap.servers")
+            .displayName("Bootstrap Servers")
+            .description("Comma-separated list of Kafka Bootstrap Servers in 
the format host:port. Mapped to Kafka bootstrap.servers")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor SECURITY_PROTOCOL = new 
PropertyDescriptor.Builder()
+            .name("security.protocol")
+            .displayName("Security Protocol")
+            .description("Security protocol used to communicate with brokers. 
Corresponds to Kafka Client security.protocol property")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues(SecurityProtocol.values())
+            .defaultValue(SecurityProtocol.PLAINTEXT.name())
+            .build();
+
+    public static final PropertyDescriptor SASL_MECHANISM = new 
PropertyDescriptor.Builder()
+            .name("sasl.mechanism")
+            .displayName("SASL Mechanism")
+            .description("SASL mechanism used for authentication. Corresponds 
to Kafka Client sasl.mechanism property")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues(SaslMechanism.getAvailableSaslMechanisms())
+            .defaultValue(SaslMechanism.GSSAPI.getValue())
+            .build();
+
+    public static final PropertyDescriptor SASL_USERNAME = new 
PropertyDescriptor.Builder()
+            .name("sasl.username")
+            .displayName("Username")
+            .description("Username provided with configured password when 
using PLAIN or SCRAM SASL Mechanisms")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .dependsOn(
+                    SASL_MECHANISM,
+                    SaslMechanism.PLAIN.getValue(),
+                    SaslMechanism.SCRAM_SHA_256.getValue(),
+                    SaslMechanism.SCRAM_SHA_512.getValue()
+            )
+            .build();
+
+    public static final PropertyDescriptor SASL_PASSWORD = new 
PropertyDescriptor.Builder()
+            .name("sasl.password")
+            .displayName("Password")
+            .description("Password provided with configured username when 
using PLAIN or SCRAM SASL Mechanisms")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .dependsOn(
+                    SASL_MECHANISM,
+                    SaslMechanism.PLAIN.getValue(),
+                    SaslMechanism.SCRAM_SHA_256.getValue(),
+                    SaslMechanism.SCRAM_SHA_512.getValue()
+            )
+            .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("ssl.context.service")
+            .displayName("SSL Context Service")
+            .description("Service supporting SSL communication with Kafka 
brokers")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    public static final PropertyDescriptor MAX_POLL_RECORDS = new 
PropertyDescriptor.Builder()
+            .name("max.poll.records")
+            .displayName("Max Poll Records")
+            .description("Specifies the maximum number of records Kafka should 
return in a single poll.")
+            .required(false)
+            .defaultValue("10000")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CLIENT_TIMEOUT = new 
PropertyDescriptor.Builder()
+            .name("default.api.timeout.ms")
+            .displayName("Client Timeout")
+            .description("Default timeout for Kafka client operations. Mapped 
to Kafka default.api.timeout.ms. The Kafka request.timeout.ms property is 
derived from half of the configured timeout")
+            .defaultValue("60 s")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor HONOR_TRANSACTIONS = new 
PropertyDescriptor.Builder()
+            .name("honor-transactions")
+            .displayName("Honor Transactions")
+            .description("Specifies whether or not NiFi should honor 
transactional guarantees when communicating with Kafka. If false, the Processor 
will use an \"isolation level\" of "
+                    + "read_uncomitted. This means that messages will be 
received as soon as they are written to Kafka but will be pulled, even if the 
producer cancels the transactions. If "
+                    + "this value is true, NiFi will not receive any messages 
for which the producer's transaction was canceled, but this can result in some 
latency since the consumer must wait "
+                    + "for the producer to finish its entire transaction 
instead of pulling as the messages become available.")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor METADATA_WAIT_TIME = new 
PropertyDescriptor.Builder()
+            .name("max.block.ms")
+            .displayName("Max Metadata Wait Time")
+            .description("The amount of time publisher will wait to obtain 
metadata or wait for the buffer to flush during the 'send' call before failing 
the "
+                    + "entire 'send' call. Corresponds to Kafka's 
'max.block.ms' property")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .defaultValue("5 sec")
+            .build();
+
+    public static final PropertyDescriptor ACK_WAIT_TIME = new 
PropertyDescriptor.Builder()
+            .name("ack.wait.time")
+            .displayName("Acknowledgment Wait Time")
+            .description("After sending a message to Kafka, this indicates the 
amount of time that we are willing to wait for a response from Kafka. "
+                    + "If Kafka does not acknowledge the message within this 
time period, the FlowFile will be routed to 'failure'.")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .defaultValue("5 secs")
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = 
Collections.unmodifiableList(Arrays.asList(
+            BOOTSTRAP_SERVERS,
+            SECURITY_PROTOCOL,
+            SASL_MECHANISM,
+            SASL_USERNAME,
+            SASL_PASSWORD,
+            SSL_CONTEXT_SERVICE,
+            MAX_POLL_RECORDS,
+            CLIENT_TIMEOUT,
+            HONOR_TRANSACTIONS,
+            METADATA_WAIT_TIME,
+            ACK_WAIT_TIME
+            //AWS_PROFILE_NAME,  // defer for now
+            //KERBEROS_SERVICE_NAME,  // defer for now
+            //SELF_CONTAINED_KERBEROS_USER_SERVICE,  // defer for now
+            //TOKEN_AUTHENTICATION,  // defer for now
+    ));
+
+    private static final Duration VERIFY_TIMEOUT = Duration.ofSeconds(2);
+
+    private static final String CONNECTION_STEP = "Kafka Broker Connection";
+
+    private static final String TOPIC_LISTING_STEP = "Kafka Topic Listing";
+
+    private Properties clientProperties;
+
+    private Properties consumerProperties;
+
+    private ServiceConfiguration serviceConfiguration;
+
+    private Kafka3ConsumerService consumerService;
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext configurationContext) {
+        clientProperties = getClientProperties(configurationContext);
+        consumerProperties = getConsumerProperties(configurationContext, 
clientProperties);
+        serviceConfiguration = getServiceConfiguration(configurationContext);
+        consumerService = new Kafka3ConsumerService(getLogger(), 
consumerProperties);
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        if (consumerService == null) {
+            getLogger().warn("Consumer Service not configured");
+        } else {
+            consumerService.close();
+        }
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public KafkaConsumerService getConsumerService(final ConsumerConfiguration 
consumerConfiguration) {
+        return consumerService;
+    }
+
+    @Override
+    public KafkaProducerService getProducerService(final ProducerConfiguration 
producerConfiguration) {
+        final Properties propertiesProducer = new Properties();
+        propertiesProducer.putAll(clientProperties);
+        if (producerConfiguration.getUseTransactions()) {
+            propertiesProducer.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG,
+                    new 
TransactionIdSupplier(producerConfiguration.getTransactionIdPrefix()).get());
+        }
+        if (producerConfiguration.getDeliveryGuarantee() != null) {
+            propertiesProducer.put(ProducerConfig.ACKS_CONFIG, 
producerConfiguration.getDeliveryGuarantee());
+        }
+        if (producerConfiguration.getCompressionCodec() != null) {
+            propertiesProducer.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, 
producerConfiguration.getCompressionCodec());
+        }
+        final String partitionClass = 
producerConfiguration.getPartitionClass();
+        if (partitionClass != null && 
partitionClass.startsWith("org.apache.kafka")) {
+            propertiesProducer.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, 
partitionClass);
+        }
+        return new Kafka3ProducerService(propertiesProducer, 
serviceConfiguration, producerConfiguration);
+    }
+
+    @Override
+    public List<ConfigVerificationResult> verify(final ConfigurationContext 
configurationContext, final ComponentLog verificationLogger, final Map<String, 
String> variables) {
+        final List<ConfigVerificationResult> results = new ArrayList<>();
+
+        final Properties clientProperties = 
getClientProperties(configurationContext);
+        clientProperties.putAll(variables);
+        try (final Admin admin = Admin.create(clientProperties)) {
+            final ListTopicsResult listTopicsResult = admin.listTopics();
+
+            final KafkaFuture<Collection<TopicListing>> requestedListings = 
listTopicsResult.listings();
+            final Collection<TopicListing> topicListings = 
requestedListings.get(VERIFY_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS);
+            final String topicListingExplanation = String.format("Topics Found 
[%d]", topicListings.size());
+            results.add(
+                    new ConfigVerificationResult.Builder()
+                            .verificationStepName(TOPIC_LISTING_STEP)
+                            .outcome(SUCCESSFUL)
+                            .explanation(topicListingExplanation)
+                            .build()
+            );
+        } catch (final Exception e) {
+            verificationLogger.error("Kafka Broker verification failed", e);
+            results.add(
+                    new ConfigVerificationResult.Builder()
+                            .verificationStepName(CONNECTION_STEP)
+                            .outcome(FAILED)
+                            .explanation(e.getMessage())
+                            .build()
+            );
+        }
+
+        return results;
+    }
+
+    private Properties getConsumerProperties(final PropertyContext 
propertyContext, final Properties defaultProperties) {
+        final Properties properties = new Properties();
+        properties.putAll(defaultProperties);
+
+        // since config for ConsumerPool is locked in at 
ControllerService.enable(),
+        final boolean honorTransactions = 
propertyContext.getProperty(HONOR_TRANSACTIONS).asBoolean();
+        if (honorTransactions) {
+            properties.put("isolation.level", "read_committed");
+        } else {
+            properties.put("isolation.level", "read_uncommitted");

Review Comment:
   If these isolation level values are used elsewhere, it would be helpful to 
define them as static strings, or perhaps in an `enum`.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/producer/ProducerCallback.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.service.producer;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.service.api.producer.FlowFileResult;
+import org.apache.nifi.kafka.service.api.producer.ProducerRecordMetadata;
+import org.apache.nifi.kafka.shared.util.Notifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+
+public class ProducerCallback implements Callback {
+    private final Logger logger = LoggerFactory.getLogger(getClass());
+
+    private final AtomicLong sentCount;
+    private final AtomicLong acknowledgedCount;
+    private final AtomicLong failedCount;
+    private final FlowFile flowFile;
+    private final List<ProducerRecordMetadata> metadatas;
+    private final List<Exception> exceptions;
+    private final Notifier notifier;
+
+    public List<Exception> getExceptions() {
+        return exceptions;
+    }
+
+    public boolean isFailure() {
+        return !exceptions.isEmpty();
+    }
+
+    public ProducerCallback(final FlowFile flowFile) {
+        this.sentCount = new AtomicLong(0L);
+        this.acknowledgedCount = new AtomicLong(0L);
+        this.failedCount = new AtomicLong(0L);
+        this.flowFile = flowFile;
+        this.metadatas = new ArrayList<>();
+        this.exceptions = new ArrayList<>();
+        this.notifier = new Notifier();
+    }
+
+    public long send() {
+        return sentCount.incrementAndGet();
+    }
+
+    @Override
+    public void onCompletion(final RecordMetadata metadata, final Exception 
exception) {
+
+        // the source `FlowFile` and the associated `RecordMetadata` need to 
somehow be associated...

Review Comment:
   Should this comment be removed?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/producer/txn/KafkaTransactionalProducerWrapper.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.service.producer.txn;
+
+import org.apache.kafka.clients.producer.Producer;
+
+public class KafkaTransactionalProducerWrapper extends KafkaProducerWrapper {
+
+    public KafkaTransactionalProducerWrapper(final Producer<byte[], byte[]> 
producer) {
+        super(producer);
+    }
+
+    @Override
+    public void init() {
+        producer.initTransactions();
+        producer.beginTransaction();
+    }
+
+    @Override
+    public void commit() {
+        try {
+            producer.commitTransaction();
+            logger.trace("committed");

Review Comment:
   This seems too generic to be useful.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.processors;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.processors.common.KafkaUtils;
+import org.apache.nifi.kafka.processors.consumer.OffsetTracker;
+import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
+import org.apache.nifi.kafka.processors.consumer.bundle.ByteRecordBundler;
+import 
org.apache.nifi.kafka.processors.consumer.convert.FlowFileStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.processors.consumer.convert.KafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.RecordStreamKafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.consumer.PollingContext;
+import org.apache.nifi.kafka.service.api.record.ByteRecord;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.KeyFormat;
+import org.apache.nifi.kafka.shared.property.OutputStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.StringUtils;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"kafka", "consumer", "record"})
+public class ConsumeKafka extends AbstractProcessor implements 
VerifiableProcessor {
+
+    static final AllowableValue TOPIC_NAME = new AllowableValue("names", 
"names", "Topic is a full topic name or comma separated list of names");
+    static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", 
"pattern", "Topic is a regex using the Java Pattern syntax");
+
+    static final PropertyDescriptor CONNECTION_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("Kafka Connection Service")
+            .displayName("Kafka Connection Service")
+            .description("Provides connections to Kafka Broker for publishing 
Kafka Records")
+            .identifiesControllerService(KafkaConnectionService.class)
+            .expressionLanguageSupported(NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
+            .name("Group ID")

Review Comment:
   This is a good question. I do not anticipate that it will be easy to migrate 
from existing Processors. For that reason, I think new property names should 
follow the general pattern of using the human-readable name as much as 
possible. The exception to that rule might be where a NiFi property name is 
exactly equivalent to a Kafka property name.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java:
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.service;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.security.plain.PlainLoginModule;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.VerifiableControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.ServiceConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.producer.KafkaProducerService;
+import org.apache.nifi.kafka.service.api.producer.ProducerConfiguration;
+import org.apache.nifi.kafka.service.consumer.Kafka3ConsumerService;
+import org.apache.nifi.kafka.service.producer.Kafka3ProducerService;
+import org.apache.nifi.kafka.shared.property.SaslMechanism;
+import org.apache.nifi.kafka.shared.property.provider.KafkaPropertyProvider;
+import 
org.apache.nifi.kafka.shared.property.provider.StandardKafkaPropertyProvider;
+import org.apache.nifi.kafka.shared.transaction.TransactionIdSupplier;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_LOCATION;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEYSTORE_TYPE;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_KEY_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_LOCATION;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_PASSWORD;
+import static 
org.apache.nifi.kafka.shared.property.KafkaClientProperty.SSL_TRUSTSTORE_TYPE;
+
+public class Kafka3ConnectionService extends AbstractControllerService 
implements KafkaConnectionService, VerifiableControllerService {
+
+    public static final PropertyDescriptor BOOTSTRAP_SERVERS = new 
PropertyDescriptor.Builder()
+            .name("bootstrap.servers")
+            .displayName("Bootstrap Servers")
+            .description("Comma-separated list of Kafka Bootstrap Servers in 
the format host:port. Mapped to Kafka bootstrap.servers")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor SECURITY_PROTOCOL = new 
PropertyDescriptor.Builder()
+            .name("security.protocol")
+            .displayName("Security Protocol")
+            .description("Security protocol used to communicate with brokers. 
Corresponds to Kafka Client security.protocol property")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues(SecurityProtocol.values())
+            .defaultValue(SecurityProtocol.PLAINTEXT.name())
+            .build();
+
+    public static final PropertyDescriptor SASL_MECHANISM = new 
PropertyDescriptor.Builder()
+            .name("sasl.mechanism")
+            .displayName("SASL Mechanism")
+            .description("SASL mechanism used for authentication. Corresponds 
to Kafka Client sasl.mechanism property")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues(SaslMechanism.getAvailableSaslMechanisms())
+            .defaultValue(SaslMechanism.GSSAPI.getValue())
+            .build();
+
+    public static final PropertyDescriptor SASL_USERNAME = new 
PropertyDescriptor.Builder()
+            .name("sasl.username")
+            .displayName("Username")
+            .description("Username provided with configured password when 
using PLAIN or SCRAM SASL Mechanisms")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .dependsOn(
+                    SASL_MECHANISM,
+                    SaslMechanism.PLAIN.getValue(),
+                    SaslMechanism.SCRAM_SHA_256.getValue(),
+                    SaslMechanism.SCRAM_SHA_512.getValue()
+            )
+            .build();
+
+    public static final PropertyDescriptor SASL_PASSWORD = new 
PropertyDescriptor.Builder()
+            .name("sasl.password")
+            .displayName("Password")
+            .description("Password provided with configured username when 
using PLAIN or SCRAM SASL Mechanisms")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .dependsOn(
+                    SASL_MECHANISM,
+                    SaslMechanism.PLAIN.getValue(),
+                    SaslMechanism.SCRAM_SHA_256.getValue(),
+                    SaslMechanism.SCRAM_SHA_512.getValue()
+            )
+            .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("ssl.context.service")
+            .displayName("SSL Context Service")
+            .description("Service supporting SSL communication with Kafka 
brokers")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    public static final PropertyDescriptor MAX_POLL_RECORDS = new 
PropertyDescriptor.Builder()
+            .name("max.poll.records")
+            .displayName("Max Poll Records")
+            .description("Specifies the maximum number of records Kafka should 
return in a single poll.")
+            .required(false)
+            .defaultValue("10000")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CLIENT_TIMEOUT = new 
PropertyDescriptor.Builder()
+            .name("default.api.timeout.ms")
+            .displayName("Client Timeout")
+            .description("Default timeout for Kafka client operations. Mapped 
to Kafka default.api.timeout.ms. The Kafka request.timeout.ms property is 
derived from half of the configured timeout")
+            .defaultValue("60 s")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .build();
+
+    public static final PropertyDescriptor HONOR_TRANSACTIONS = new 
PropertyDescriptor.Builder()
+            .name("honor-transactions")
+            .displayName("Honor Transactions")
+            .description("Specifies whether or not NiFi should honor 
transactional guarantees when communicating with Kafka. If false, the Processor 
will use an \"isolation level\" of "
+                    + "read_uncomitted. This means that messages will be 
received as soon as they are written to Kafka but will be pulled, even if the 
producer cancels the transactions. If "
+                    + "this value is true, NiFi will not receive any messages 
for which the producer's transaction was canceled, but this can result in some 
latency since the consumer must wait "
+                    + "for the producer to finish its entire transaction 
instead of pulling as the messages become available.")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor METADATA_WAIT_TIME = new 
PropertyDescriptor.Builder()
+            .name("max.block.ms")
+            .displayName("Max Metadata Wait Time")
+            .description("The amount of time publisher will wait to obtain 
metadata or wait for the buffer to flush during the 'send' call before failing 
the "
+                    + "entire 'send' call. Corresponds to Kafka's 
'max.block.ms' property")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .defaultValue("5 sec")
+            .build();
+
+    public static final PropertyDescriptor ACK_WAIT_TIME = new 
PropertyDescriptor.Builder()
+            .name("ack.wait.time")
+            .displayName("Acknowledgment Wait Time")
+            .description("After sending a message to Kafka, this indicates the 
amount of time that we are willing to wait for a response from Kafka. "
+                    + "If Kafka does not acknowledge the message within this 
time period, the FlowFile will be routed to 'failure'.")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(true)
+            .defaultValue("5 secs")
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = 
Collections.unmodifiableList(Arrays.asList(
+            BOOTSTRAP_SERVERS,
+            SECURITY_PROTOCOL,
+            SASL_MECHANISM,
+            SASL_USERNAME,
+            SASL_PASSWORD,
+            SSL_CONTEXT_SERVICE,
+            MAX_POLL_RECORDS,
+            CLIENT_TIMEOUT,
+            HONOR_TRANSACTIONS,
+            METADATA_WAIT_TIME,
+            ACK_WAIT_TIME
+            //AWS_PROFILE_NAME,  // defer for now
+            //KERBEROS_SERVICE_NAME,  // defer for now

Review Comment:
   I agree with deferring the implementation of these strategies, but I 
recommend removing the commented lines from the code and instead write them up 
in a separate Jira issue.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.processors;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.processors.common.KafkaUtils;
+import org.apache.nifi.kafka.processors.consumer.OffsetTracker;
+import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
+import org.apache.nifi.kafka.processors.consumer.bundle.ByteRecordBundler;
+import 
org.apache.nifi.kafka.processors.consumer.convert.FlowFileStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.processors.consumer.convert.KafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.RecordStreamKafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.consumer.PollingContext;
+import org.apache.nifi.kafka.service.api.record.ByteRecord;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.KeyFormat;
+import org.apache.nifi.kafka.shared.property.OutputStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.StringUtils;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"kafka", "consumer", "record"})
+public class ConsumeKafka extends AbstractProcessor implements 
VerifiableProcessor {
+
+    static final AllowableValue TOPIC_NAME = new AllowableValue("names", 
"names", "Topic is a full topic name or comma separated list of names");
+    static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", 
"pattern", "Topic is a regex using the Java Pattern syntax");
+
+    static final PropertyDescriptor CONNECTION_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("Kafka Connection Service")
+            .displayName("Kafka Connection Service")
+            .description("Provides connections to Kafka Broker for publishing 
Kafka Records")
+            .identifiesControllerService(KafkaConnectionService.class)
+            .expressionLanguageSupported(NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
+            .name("Group ID")
+            .displayName("Group ID")
+            .description("Kafka Consumer Group Identifier corresponding to 
Kafka group.id property")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(NONE)
+            .build();
+
+    static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
+            .name("topic")
+            .displayName("Topic Name(s)")

Review Comment:
   Recommend avoiding the parentheses and just using `Topic Names`:
   ```suggestion
               .displayName("Topic Names")
   ```



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * 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.processors;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.kafka.processors.common.KafkaUtils;
+import org.apache.nifi.kafka.processors.consumer.OffsetTracker;
+import org.apache.nifi.kafka.processors.consumer.ProcessingStrategy;
+import org.apache.nifi.kafka.processors.consumer.bundle.ByteRecordBundler;
+import 
org.apache.nifi.kafka.processors.consumer.convert.FlowFileStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.processors.consumer.convert.KafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.RecordStreamKafkaMessageConverter;
+import 
org.apache.nifi.kafka.processors.consumer.convert.WrapperRecordStreamKafkaMessageConverter;
+import org.apache.nifi.kafka.service.api.KafkaConnectionService;
+import org.apache.nifi.kafka.service.api.common.PartitionState;
+import org.apache.nifi.kafka.service.api.consumer.AutoOffsetReset;
+import org.apache.nifi.kafka.service.api.consumer.ConsumerConfiguration;
+import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService;
+import org.apache.nifi.kafka.service.api.consumer.PollingContext;
+import org.apache.nifi.kafka.service.api.record.ByteRecord;
+import org.apache.nifi.kafka.shared.attribute.KafkaFlowFileAttribute;
+import org.apache.nifi.kafka.shared.property.KeyEncoding;
+import org.apache.nifi.kafka.shared.property.KeyFormat;
+import org.apache.nifi.kafka.shared.property.OutputStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.VerifiableProcessor;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.StringUtils;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"kafka", "consumer", "record"})
+public class ConsumeKafka extends AbstractProcessor implements 
VerifiableProcessor {
+
+    static final AllowableValue TOPIC_NAME = new AllowableValue("names", 
"names", "Topic is a full topic name or comma separated list of names");
+    static final AllowableValue TOPIC_PATTERN = new AllowableValue("pattern", 
"pattern", "Topic is a regex using the Java Pattern syntax");
+
+    static final PropertyDescriptor CONNECTION_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("Kafka Connection Service")
+            .displayName("Kafka Connection Service")
+            .description("Provides connections to Kafka Broker for publishing 
Kafka Records")
+            .identifiesControllerService(KafkaConnectionService.class)
+            .expressionLanguageSupported(NONE)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
+            .name("Group ID")
+            .displayName("Group ID")
+            .description("Kafka Consumer Group Identifier corresponding to 
Kafka group.id property")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(NONE)
+            .build();
+
+    static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder()
+            .name("topic")
+            .displayName("Topic Name(s)")
+            .description("The name of the Kafka Topic(s) from which the 
Processor consumes Kafka Records. More than one can be supplied if comma 
separated.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+            .build();
+
+    static final PropertyDescriptor TOPIC_TYPE = new 
PropertyDescriptor.Builder()
+            .name("topic_type")
+            .displayName("Topic Name Format")
+            .description("Specifies whether the Topic(s) provided are a comma 
separated list of names or a single regular expression")
+            .required(true)
+            .allowableValues(TOPIC_NAME, TOPIC_PATTERN)
+            .defaultValue(TOPIC_NAME)
+            .build();
+
+    static final PropertyDescriptor RECORD_READER = new 
PropertyDescriptor.Builder()
+            .name("record-reader")
+            .displayName("Record Reader")

Review Comment:
   ```suggestion
               .name("Record Reader")
               .displayName("Record Reader")
   ```



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-3-service/src/main/java/org/apache/nifi/kafka/service/producer/ProducerCallback.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.service.producer;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.kafka.service.api.producer.FlowFileResult;
+import org.apache.nifi.kafka.service.api.producer.ProducerRecordMetadata;
+import org.apache.nifi.kafka.shared.util.Notifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
+
+public class ProducerCallback implements Callback {
+    private final Logger logger = LoggerFactory.getLogger(getClass());
+
+    private final AtomicLong sentCount;
+    private final AtomicLong acknowledgedCount;
+    private final AtomicLong failedCount;
+    private final FlowFile flowFile;
+    private final List<ProducerRecordMetadata> metadatas;
+    private final List<Exception> exceptions;
+    private final Notifier notifier;
+
+    public List<Exception> getExceptions() {
+        return exceptions;
+    }
+
+    public boolean isFailure() {
+        return !exceptions.isEmpty();
+    }
+
+    public ProducerCallback(final FlowFile flowFile) {
+        this.sentCount = new AtomicLong(0L);
+        this.acknowledgedCount = new AtomicLong(0L);
+        this.failedCount = new AtomicLong(0L);
+        this.flowFile = flowFile;
+        this.metadatas = new ArrayList<>();
+        this.exceptions = new ArrayList<>();
+        this.notifier = new Notifier();
+    }
+
+    public long send() {
+        return sentCount.incrementAndGet();
+    }
+
+    @Override
+    public void onCompletion(final RecordMetadata metadata, final Exception 
exception) {
+
+        // the source `FlowFile` and the associated `RecordMetadata` need to 
somehow be associated...
+
+        if (exception == null) {
+            acknowledgedCount.addAndGet(1L);
+            metadatas.add(toProducerRecordMetadata(metadata));
+        } else {
+            failedCount.addAndGet(1L);
+            exceptions.add(exception);
+        }
+        logger.trace("NIFI-11259 - onCompletion() - [{}][{}][{}][{}]",
+                metadata, exception, sentCount.get(), acknowledgedCount.get());

Review Comment:
   It looks like this should be removed.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to