You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/06/21 19:51:56 UTC

[GitHub] [nifi] markap14 commented on a diff in pull request #6131: NIFI-9822 - ConsumeKafkaRecord allows writing out Kafka record key

markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r902981711


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java:
##########
@@ -212,6 +217,31 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
+    static final PropertyDescriptor CONSUME_STRATEGY = new PropertyDescriptor.Builder()
+            .name("consume-strategy")
+            .displayName("Consume Strategy")

Review Comment:
   This doesn't really control how we consume data from Kafka. Rather, it controls how we write out the data. So perhaps "Output Strategy" is more suitable?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -271,7 +273,21 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
-
+    static final PropertyDescriptor PUBLISH_STRATEGY = new PropertyDescriptor.Builder()

Review Comment:
   With the addition of this property, I think some of the existing properties can also be hidden if using the wrapper. Specifically, just like in the consumer, we should make MESSAGE_KEY_FIELD and ATTRIBUTE_NAME_REGEX dependent properties because it doesn't make sense to specify those if using the Wrapper.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java:
##########
@@ -110,6 +110,12 @@ public final class KafkaProcessorUtils {
         "When unable to publish a FlowFile to Kafka, the FlowFile will be placed back on the top of its queue so that it will be the next FlowFile tried again. " +
             "For dataflows where ordering of FlowFiles is important, this strategy can be used along with ensuring that the each processor in the dataflow uses only a single Concurrent Task.");
 
+    static final AllowableValue WRITE_VALUE_ONLY = new AllowableValue("write-value-only", "Write Value Only", "Write only the Kafka Record value.");
+    static final AllowableValue USE_WRAPPER = new AllowableValue("use-wrapper", "Use Wrapper", "Write the Kafka Record key, value, headers, and metadata.");

Review Comment:
   Perhaps we should mention in the description explicitly that these fields will be wrapped in a wrapper element, and to see Processor Usage for more information.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_6/additionalDetails.html:
##########
@@ -256,5 +256,69 @@ <h3>SASL_SSL</h3>
             ssl.client.auth property.
         </p>
 
+        <h2>Output Modes</h2>
+        <div>
+            <p>This processor (NiFi 1.17+) offers multiple output strategies (configured via processor property 'Consume
+                Strategy') for converting Kafka records into FlowFiles.</p>
+            <ul>
+                <li>Consume Strategy 'Write Value Only' (the default) emits flowfile records containing only the Kafka
+                    record value.
+                </li>
+                <li>Consume Strategy 'Use Wrapper' (new) emits flowfile records containing the Kafka record key, value,
+                    and headers, as well as additional metadata from the Kafka record.
+                </li>
+            </ul>
+
+
+            <p>The record schema that is used when 'Use Wrapper' is active is as follows:</p>
+            <ul>
+                <li>key: one of RECORD, STRING, or BYTE_ARRAY (based on processor configuration property 'Key Format')
+                </li>
+                <li>value: RECORD</li>
+                <li>headers: MAP (STRING, STRING)</li>
+                <li>metadata: RECORD
+                    <ul>
+                        <li>topic: STRING</li>
+                        <li>partition: INTEGER</li>
+                        <li>offset: LONG</li>
+                        <li>timestamp: TIMESTAMP</li>
+                    </ul>
+                </li>
+            </ul>
+
+            <p>If the Consume Strategy property 'Use Wrapper' is active, an additional processor configuration property
+                ('Key Format') is activated. This property is used to fine-tune the transformation of the incoming Kafka
+                record. The possible values for 'Key Format' are 'Byte Array', 'String', or 'Record'.</p>
+            <ul>
+                <li>'Byte Array' supplies the Kafka Record Key bytes unchanged from the incoming Kafka record.</li>
+                <li>'String' converts the Kafka Record Key bytes into a string using the UTF-8 character encoding.
+                    (Failure to parse the key bytes as UTF-8 will result in the record being routed to the
+                    'parse.failure' relationship.)
+                </li>
+                <li>'Record' converts the Kafka Record Key bytes into a deserialized NiFi record, using the associated
+                    'Key Record Reader' controller service.
+                </li>
+            </ul>
+
+            <p>If the Key Format property is set to 'Record', an additional processor configuration property is
+                activated.</p>
+            <ul>
+                <li>'Key Record Reader' is used to specify the controller service that is used to deserialize the key
+                    bytes. It may be set to any available implementation of the NiFi 'RecordReaderFactory' interface.
+                </li>
+            </ul>
+
+            <p>These new processor properties may be used to extend the capabilities of ConsumeKafkaRecord_2_6, by
+                optionally incorporating additional information from the Kafka record (key, headers, metadata) into the
+                outbound flowfile. And the Kafka key data may now be interpreted as a record, rather than as a string,
+                enabling additional decision-making by downstream processors in your flow.</p>
+
+            <p>Additionally, the choice of the 'Consume Strategy' processor property affects the related properties
+                'Headers to Add as Attributes (Regex)' and 'Key Attribute Encoding'. Since Output Strategy 'Use
+                Wrapper' includes headers and keys in the FlowFile content, they are not also added to the FlowFile
+                attributes. These properties are available only when the FlowFile Output Strategy is set to 'Write
+                Value Only'.</p>
+        </div>

Review Comment:
   I'd recommend providing an example output that was written using the JsonRecordSetWriter. This will help to clarify exactly what would be produced by the processor.



##########
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:
##########
@@ -189,5 +189,30 @@ <h3>SASL_SSL</h3>
             See the SSL section for a description of how to configure the SSL Context Service based on the
             ssl.client.auth property.
         </p>
+        <h2>Record Key Output</h2>
+        <div>
+            <p>This processor includes optional properties that control the serialization of the key into the target
+                Kafka record.</p>
+            <ul>
+                <li>'Publish Strategy'</li>
+                <li>'Record Key Writer'</li>
+            </ul>
+
+            <p>'Publish Strategy' controls the mode used to convert the FlowFile record into a Kafka record.</p>
+            <ul>
+                <li>'Write Value Only' (the default) persists the record key as a byte array.</li>
+                <li>'Use Wrapper' persists the record key as a record.</li>
+            </ul>
+
+            <p>If Publish Strategy 'Use Wrapper' is enabled, an additional processor configuration property is
+                activated: 'Record Key Writer'.</p>
+
+            <p>'Record Key Writer', if enabled, serializes the NiFi record key field into the Kafka record using the
+                specified implementation of the NiFi 'RecordSetWriter' interface. This may be used to emit the key field
+                as JSON, Avro, XML, or some other data format, where it may be used in the decision-making process of
+                downstream data processing (including that available in ConsumeKafkaRecord_2_6). If not defined, the
+                default implementation serializes the record key as an unmodified byte array (unchanged from previous
+                versions of the processor).</p>
+        </div>

Review Comment:
   Would recommend showing the same example that I referred to in ConsumeKafkaRecord_2_6, where the payload is in JSON, and then showing exactly what would be sent to Kafka, what would be used as the key, and the headers. For the metadata, we have a field for the topic also. Would that be used here? Would any of the 'metadata' fields be used? Or would they all be ignored?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java:
##########
@@ -212,6 +217,31 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
+    static final PropertyDescriptor CONSUME_STRATEGY = new PropertyDescriptor.Builder()
+            .name("consume-strategy")
+            .displayName("Consume Strategy")
+            .description("The format used to consume the Kafka record into a FlowFile record.")
+            .required(true)
+            .defaultValue(WRITE_VALUE_ONLY.getValue())
+            .allowableValues(WRITE_VALUE_ONLY, USE_WRAPPER)
+            .build();
+    static final PropertyDescriptor KEY_FORMAT = new PropertyDescriptor.Builder()
+            .name("key-format")
+            .displayName("Key Format")
+            .description("The format used to consume the Kafka record key into the incoming FlowFile record.")
+            .required(true)
+            .defaultValue(KEY_AS_BYTE_ARRAY.getValue())
+            .allowableValues(KEY_AS_STRING, KEY_AS_BYTE_ARRAY, KEY_AS_RECORD)
+            .dependsOn(CONSUME_STRATEGY, USE_WRAPPER)
+            .build();
+    static final PropertyDescriptor KEY_RECORD_READER = new PropertyDescriptor.Builder()
+            .name("key-record-reader")
+            .displayName("Key Record Reader")
+            .description("The Key Record Reader to use for incoming FlowFiles")

Review Comment:
   There are no incoming FlowFiles for this processor. An incoming FlowFile would imply that a FlowFile is read from an incoming queue. Perhaps "The Record Reader to use for parsing the Kafka Record's key into a Record"?



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -271,7 +273,21 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
-
+    static final PropertyDescriptor PUBLISH_STRATEGY = new PropertyDescriptor.Builder()
+            .name("publish-strategy")
+            .displayName("Publish Strategy")
+            .description("The format used to publish the outgoing FlowFile record to Kafka.")
+            .required(true)
+            .defaultValue(WRITE_VALUE_ONLY.getValue())
+            .allowableValues(WRITE_VALUE_ONLY, USE_WRAPPER)

Review Comment:
   I'm not sure that "Write Value Only" is applicable here. WIth this strategy, we can still send message headers using attributes, and a message key by specifying a field name.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java:
##########
@@ -212,6 +217,31 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
+    static final PropertyDescriptor CONSUME_STRATEGY = new PropertyDescriptor.Builder()
+            .name("consume-strategy")
+            .displayName("Consume Strategy")
+            .description("The format used to consume the Kafka record into a FlowFile record.")
+            .required(true)
+            .defaultValue(WRITE_VALUE_ONLY.getValue())
+            .allowableValues(WRITE_VALUE_ONLY, USE_WRAPPER)
+            .build();
+    static final PropertyDescriptor KEY_FORMAT = new PropertyDescriptor.Builder()
+            .name("key-format")
+            .displayName("Key Format")
+            .description("The format used to consume the Kafka record key into the incoming FlowFile record.")

Review Comment:
   This is also not really related to consuming - we consume by requesting a ConsumerRecord from the Kafka client. This is about how to write the key. Perhaps we should say something along the lines of "Specifies how to represent the Kafka Record's Key in the output"



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -271,7 +273,21 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
-
+    static final PropertyDescriptor PUBLISH_STRATEGY = new PropertyDescriptor.Builder()
+            .name("publish-strategy")
+            .displayName("Publish Strategy")
+            .description("The format used to publish the outgoing FlowFile record to Kafka.")

Review Comment:
   I would avoid the term "outgoing FlowFile" because that term is generally used to refer to a FlowFile that is being transferred to an outbound relationship. Here, it's not an outgoing FlowFile that we're publishing to Kafka. It's an incoming FlowFile.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafkaMock.java:
##########
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.kafka.pubsub;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.header.Headers;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.regex.Pattern;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestPublishKafkaMock {
+
+    private static long ordinal = 0L;
+
+    /**
+     * JSON serialization helper.
+     */
+    private final ObjectMapper mapper = new ObjectMapper();
+
+    /**
+     * Ensure fresh data for each test run.
+     */
+    private static final long TIMESTAMP = System.currentTimeMillis();
+
+    /**
+     * The name of the test kafka topic to be created.
+     */
+    private static final String TEST_TOPIC_PUBLISH = "nifi-publish-" + TIMESTAMP;
+
+
+    @Test
+    public void testPublishRecordNullKey() throws JsonProcessingException, InitializationException {
+        // create flowfile to publish
+        final Map<String, String> attributes = new TreeMap<>();
+        attributes.put("attrKeyA", "attrValueA");
+        attributes.put("attrKeyB", "attrValueB");
+        final ObjectNode node = mapper.createObjectNode().put("recordA", 1).put("recordB", "valueB");
+        final String value = mapper.writeValueAsString(node);
+        final MockFlowFile flowFile = new MockFlowFile(++ordinal);
+        flowFile.putAttributes(attributes);
+        flowFile.setData(value.getBytes(UTF_8));
+        // publish flowfile
+        final Collection<ProducerRecord<byte[], byte[]>> producedRecords = new ArrayList<>();
+        final TestRunner runner = getTestRunner(producedRecords);
+        runner.setProperty("topic", TEST_TOPIC_PUBLISH);
+        runner.setProperty("attribute-name-regex", ".*A");
+        runner.enqueue(flowFile);
+        runner.run(1);
+        // verify results
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("failure", 0);
+        assertEquals(1, producedRecords.size());
+        final ProducerRecord<byte[], byte[]> record = producedRecords.iterator().next();
+        assertEquals(TEST_TOPIC_PUBLISH, record.topic());
+        final Headers headers = record.headers();
+        assertEquals(1, headers.toArray().length);
+        assertEquals("attrValueA", new String(headers.lastHeader("attrKeyA").value(), UTF_8));
+        assertNull(record.key());
+        assertEquals(value, new String(record.value(), UTF_8));
+    }
+
+    @Test
+    public void testPublishRecordStringKey() throws JsonProcessingException, InitializationException {
+        // create flowfile to publish
+        final Map<String, String> attributes = new TreeMap<>();
+        attributes.put("attrKeyA", "attrValueA");
+        attributes.put("attrKeyB", "attrValueB");
+        attributes.put("messageKey", "this-is-a-key");
+        final ObjectNode node = mapper.createObjectNode().put("recordA", 1).put("recordB", "valueB");
+        final String value = mapper.writeValueAsString(node);
+        final MockFlowFile flowFile = new MockFlowFile(++ordinal);
+        flowFile.putAttributes(attributes);
+        flowFile.setData(value.getBytes(UTF_8));
+        // publish flowfile
+        final Collection<ProducerRecord<byte[], byte[]>> producedRecords = new ArrayList<>();
+        final TestRunner runner = getTestRunner(producedRecords);
+        runner.setProperty("topic", TEST_TOPIC_PUBLISH);
+        runner.setProperty("attribute-name-regex", ".*B");
+        runner.setProperty("message-key-field", "recordB");
+        runner.enqueue(flowFile);
+        runner.run(1);
+        // verify results
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("failure", 0);
+        assertEquals(1, producedRecords.size());
+        final ProducerRecord<byte[], byte[]> record = producedRecords.iterator().next();
+        assertEquals(TEST_TOPIC_PUBLISH, record.topic());
+        final Headers headers = record.headers();
+        assertEquals(1, headers.toArray().length);
+        assertEquals("attrValueB", new String(headers.lastHeader("attrKeyB").value(), UTF_8));
+        assertNotNull(record.key());
+        assertEquals("valueB", new String(record.key(), UTF_8));
+        assertNotNull(record.value());
+        assertEquals(value, new String(record.value(), UTF_8));
+    }
+
+    @Test
+    public void testPublishRecordWrapperStrategyNullKey() throws JsonProcessingException, InitializationException {
+        // create flowfile to publish
+        final Map<String, String> attributes = new TreeMap<>();
+        attributes.put("attrKeyA", "attrValueA");
+        attributes.put("attrKeyB", "attrValueB");
+        attributes.put("messageKey", "this-is-a-key");
+        final ObjectNode node = mapper.createObjectNode().put("recordA", 1).put("recordB", "valueB");
+        final String value = mapper.writeValueAsString(node);
+        final MockFlowFile flowFile = new MockFlowFile(++ordinal);
+        flowFile.putAttributes(attributes);
+        flowFile.setData(value.getBytes(UTF_8));
+        // publish flowfile
+        final Collection<ProducerRecord<byte[], byte[]>> producedRecords = new ArrayList<>();
+        final TestRunner runner = getTestRunner(producedRecords);
+        runner.setProperty("topic", TEST_TOPIC_PUBLISH);
+        runner.setProperty("attribute-name-regex", "attr.*");
+        runner.setProperty("publish-strategy", "use-wrapper");
+        runner.enqueue(flowFile);
+        runner.run(1);
+        // verify results
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("failure", 0);

Review Comment:
   This can be simplified as `runner.assertAllFlowFilesTransferred("success", 1);`. This ensures that 1 went to success and also that no FlowFiles went anywhere else. Not particularly important, just wanted to point it out.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -653,6 +672,81 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
         }
     }
 
+    private MapRecord toWrapperRecord(final ConsumerRecord<byte[], byte[]> consumerRecord, final Record record)
+            throws IOException, SchemaNotFoundException, MalformedRecordException {
+        final Tuple<RecordField, Object> tupleKey  = toWrapperRecordKey(consumerRecord);
+        final Tuple<RecordField, Object> tupleValue  = toWrapperRecordValue(record);
+        final Tuple<RecordField, Object> tupleHeaders  = toWrapperRecordHeaders(consumerRecord);
+        final Tuple<RecordField, Object> tupleMetadata = toWrapperRecordMetadata(consumerRecord);
+        final RecordSchema rootRecordSchema = new SimpleRecordSchema(Arrays.asList(
+                tupleKey.getKey(), tupleValue.getKey(), tupleHeaders.getKey(), tupleMetadata.getKey()));
+
+        final Map<String, Object> recordValues = new HashMap<>();
+        recordValues.put(tupleKey.getKey().getFieldName(), tupleKey.getValue());
+        recordValues.put(tupleValue.getKey().getFieldName(), tupleValue.getValue());
+        recordValues.put(tupleHeaders.getKey().getFieldName(), tupleHeaders.getValue());
+        recordValues.put(tupleMetadata.getKey().getFieldName(), tupleMetadata.getValue());
+        return new MapRecord(rootRecordSchema, recordValues);
+    }
+
+    private Tuple<RecordField, Object> toWrapperRecordKey(final ConsumerRecord<byte[], byte[]> consumerRecord)
+            throws IOException, SchemaNotFoundException, MalformedRecordException {
+        final Tuple<RecordField, Object> tuple;
+        final byte[] key = consumerRecord.key() == null ? new byte[0] : consumerRecord.key();
+        if (KafkaProcessorUtils.KEY_AS_RECORD.getValue().equals(keyFormat)) {
+            final Map<String, String> attributes = getAttributes(consumerRecord);
+            try (final InputStream is = new ByteArrayInputStream(key)) {
+                try (final RecordReader reader = keyReaderFactory.createRecordReader(attributes, is, key.length, logger)) {
+                    final Record record = reader.nextRecord();
+                    final RecordField recordField = new RecordField("key", RecordFieldType.RECORD.getRecordDataType(record.getSchema()));
+                    tuple = new Tuple<>(recordField, record);
+                }
+            }
+        } else if (KafkaProcessorUtils.KEY_AS_STRING.getValue().equals(keyFormat)) {
+            final RecordField recordField = new RecordField("key", RecordFieldType.STRING.getDataType());
+            tuple = new Tuple<>(recordField, new String(key, StandardCharsets.UTF_8));
+        } else {
+            final RecordField recordField = new RecordField("key",
+                    RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()));
+            tuple = new Tuple<>(recordField, ArrayUtils.toObject(key));
+        }
+        return tuple;
+    }
+
+    private Tuple<RecordField, Object> toWrapperRecordValue(final Record record) {
+        final RecordField recordField = new RecordField(
+                "value", RecordFieldType.RECORD.getRecordDataType(record.getSchema()));
+        return new Tuple<>(recordField, record);
+    }
+
+    private Tuple<RecordField, Object> toWrapperRecordHeaders(final ConsumerRecord<byte[], byte[]> consumerRecord) {
+        final RecordField recordField = new RecordField(
+                "headers", RecordFieldType.MAP.getMapDataType(RecordFieldType.STRING.getDataType()));
+        final Map<String, String> headers = new HashMap<>();
+        for (final Header header : consumerRecord.headers()) {
+            headers.put(header.key(), new String(header.value(), headerCharacterSet));
+        }
+        return new Tuple<>(recordField, headers);
+    }
+
+    private Tuple<RecordField, Object> toWrapperRecordMetadata(final ConsumerRecord<byte[], byte[]> consumerRecord) {
+        final RecordField fieldTopic = new RecordField("topic", RecordFieldType.STRING.getDataType());
+        final RecordField fieldPartition = new RecordField("partition", RecordFieldType.INT.getDataType());
+        final RecordField fieldOffset = new RecordField("offset", RecordFieldType.LONG.getDataType());
+        final RecordField fieldTimestamp = new RecordField("timestamp", RecordFieldType.TIMESTAMP.getDataType());
+        final RecordSchema schema = new SimpleRecordSchema(Arrays.asList(
+                fieldTopic, fieldPartition, fieldOffset, fieldTimestamp));

Review Comment:
   This schema and all of its field can be statically defined once, rather than instantiating all of this for every invocation.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.pubsub.ConsumeKafkaRecord_2_6/additionalDetails.html:
##########
@@ -256,5 +256,69 @@ <h3>SASL_SSL</h3>
             ssl.client.auth property.
         </p>
 
+        <h2>Output Modes</h2>
+        <div>
+            <p>This processor (NiFi 1.17+) offers multiple output strategies (configured via processor property 'Consume
+                Strategy') for converting Kafka records into FlowFiles.</p>
+            <ul>
+                <li>Consume Strategy 'Write Value Only' (the default) emits flowfile records containing only the Kafka
+                    record value.
+                </li>
+                <li>Consume Strategy 'Use Wrapper' (new) emits flowfile records containing the Kafka record key, value,
+                    and headers, as well as additional metadata from the Kafka record.
+                </li>
+            </ul>
+
+
+            <p>The record schema that is used when 'Use Wrapper' is active is as follows:</p>
+            <ul>
+                <li>key: one of RECORD, STRING, or BYTE_ARRAY (based on processor configuration property 'Key Format')
+                </li>
+                <li>value: RECORD</li>
+                <li>headers: MAP (STRING, STRING)</li>
+                <li>metadata: RECORD
+                    <ul>
+                        <li>topic: STRING</li>
+                        <li>partition: INTEGER</li>
+                        <li>offset: LONG</li>
+                        <li>timestamp: TIMESTAMP</li>
+                    </ul>
+                </li>
+            </ul>

Review Comment:
   Is OK to provide the schema this way, as it's fairly easy to understand. But we should also provide the schema as an Avro Schema, since that's what the user will likely need to configure



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java:
##########
@@ -193,10 +196,24 @@ void publish(final FlowFile flowFile, final RecordSet recordSet, final RecordSet
                     additionalAttributes = writeResult.getAttributes();
                     writer.flush();
                 }
-
                 final byte[] messageContent = baos.toByteArray();
-                final String key = messageKeyField == null ? null : record.getAsString(messageKeyField);
-                final byte[] messageKey = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8);
+
+                final byte[] messageKey;
+                if ((recordKeyWriterFactory == null) || (messageKeyField == null)) {
+                    messageKey = Optional.ofNullable(record.getAsString(messageKeyField))
+                            .map(s -> s.getBytes(StandardCharsets.UTF_8)).orElse(null);
+                } else {
+                    try (final ByteArrayOutputStream os = new ByteArrayOutputStream(1024)) {
+                        final Record keyRecord = Optional.ofNullable(record.getValue(messageKeyField))
+                                .filter(Record.class::isInstance).map(Record.class::cast)
+                                .orElseThrow(() ->  new IOException("The property 'Record Key Writer' is defined, but the record key is not a record"));

Review Comment:
   Not sure that it makes sense to throw an Exception in this case. If the key is a `String`, we should just convert it to a `byte[]` via `String.toByteArray(StandardCharsets.UTF_8)`. If the key is a `byte[]` already then just send it as-is. We should only use the Record Reader when necessary.
   In short - we should be explicit in what we produce, and lenient in what we accept.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestConsumeKafkaMock.java:
##########
@@ -0,0 +1,520 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.kafka.pubsub;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.node.TextNode;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static java.nio.charset.StandardCharsets.ISO_8859_1;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestConsumeKafkaMock {
+
+    /**
+     * JSON serialization helper.
+     */
+    private final ObjectMapper mapper = new ObjectMapper();
+
+    /**
+     * Kafka server endpoint (mock) for test interactions.
+     */
+    private static final String BOOTSTRAP_SERVER = "localhost:59092";
+
+    /**
+     * Ensure fresh data for each test run.
+     */
+    private static final long TIMESTAMP = System.currentTimeMillis();
+
+    /**
+     * The name of the test kafka topic to be created.
+     */
+    private static final String TEST_TOPIC = "nifi-consume-" + TIMESTAMP;
+
+    /**
+     * The name of the test kafka group to use.
+     */
+    private static final String TEST_GROUP = "nifi-group-" + TIMESTAMP;
+
+    @Test
+    public void testConsumeRecordNullKey() throws JsonProcessingException, InitializationException {
+        final ObjectNode node = mapper.createObjectNode().put("a", 1).put("b", "2");
+        final String value = mapper.writeValueAsString(node);
+        final ArrayNode nodeRecordSet = mapper.createArrayNode().add(node);
+        final String valueRecordSet = mapper.writeValueAsString(nodeRecordSet);
+        final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
+                TEST_TOPIC, 0, 0, null, value.getBytes(UTF_8));
+        final ConsumerRecords<byte[], byte[]> consumerRecords = getConsumerRecords(record);
+
+        final TestRunner runner = getTestRunner(consumerRecords, TEST_TOPIC, TEST_GROUP);
+        runner.run(1);
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("parse.failure", 0);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
+        final MockFlowFile flowFile = flowFiles.iterator().next();
+        assertEquals(valueRecordSet, flowFile.getContent());
+        assertNull(flowFile.getAttribute("kafka.key"));
+        assertEquals("0", flowFile.getAttribute("kafka.partition"));
+        assertEquals(TEST_TOPIC, flowFile.getAttribute("kafka.topic"));
+        assertEquals(TEST_GROUP, flowFile.getAttribute("kafka.consumer.id"));
+    }
+
+    @Test
+    public void testConsumeRecordTextKey() throws Exception {
+        final String key = "a-kafka-record-key";
+        final ObjectNode node = mapper.createObjectNode().put("c", 3).put("d", "4");
+        final String value = mapper.writeValueAsString(node);
+        final ArrayNode nodeRecordSet = mapper.createArrayNode().add(node);
+        final String valueRecordSet = mapper.writeValueAsString(nodeRecordSet);
+        final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
+                TEST_TOPIC, 0, 0, key.getBytes(UTF_8), value.getBytes(UTF_8));
+        final ConsumerRecords<byte[], byte[]> consumerRecords = getConsumerRecords(record);
+
+        final TestRunner runner = getTestRunner(consumerRecords, TEST_TOPIC, TEST_GROUP);
+        runner.run(1);
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("parse.failure", 0);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
+        final MockFlowFile flowFile = flowFiles.iterator().next();
+        assertEquals(valueRecordSet, flowFile.getContent());
+        assertEquals(key, flowFile.getAttribute("kafka.key"));
+        assertEquals("0", flowFile.getAttribute("kafka.partition"));
+        assertEquals(TEST_TOPIC, flowFile.getAttribute("kafka.topic"));
+        assertEquals(TEST_GROUP, flowFile.getAttribute("kafka.consumer.id"));
+    }
+
+    @Test
+    public void testConsumeRecordJsonKeyNoKeyReader() throws Exception {
+        final ObjectNode nodeKey = mapper.createObjectNode().put("key", true);
+        final String key = mapper.writeValueAsString(nodeKey);
+        final ObjectNode node = mapper.createObjectNode().put("e", 5).put("f", "6");
+        final String value = mapper.writeValueAsString(node);
+        final ArrayNode nodeRecordSet = mapper.createArrayNode().add(node);
+        final String valueRecordSet = mapper.writeValueAsString(nodeRecordSet);
+        final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
+                TEST_TOPIC, 0, 0, key.getBytes(UTF_8), value.getBytes(UTF_8));
+        final ConsumerRecords<byte[], byte[]> consumerRecords = getConsumerRecords(record);
+
+        final TestRunner runner = getTestRunner(consumerRecords, TEST_TOPIC, TEST_GROUP);
+        runner.run(1);
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("parse.failure", 0);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
+        final MockFlowFile flowFile = flowFiles.iterator().next();
+        assertEquals(valueRecordSet, flowFile.getContent());
+        assertEquals(key, flowFile.getAttribute("kafka.key"));
+        assertEquals("0", flowFile.getAttribute("kafka.partition"));
+        assertEquals(TEST_TOPIC, flowFile.getAttribute("kafka.topic"));
+        assertEquals(TEST_GROUP, flowFile.getAttribute("kafka.consumer.id"));
+    }
+
+    @Test
+    public void testConsumeRecordWrapperStrategyKeyFormatDefault() throws Exception {
+        final ObjectNode nodeToKafkaKey = mapper.createObjectNode().put("key", true);
+        final String textToKafkaKey = mapper.writeValueAsString(nodeToKafkaKey);
+        final ObjectNode nodeToKafkaValue = mapper.createObjectNode().put("g", 7).put("h", "8");
+        final String textToKafkaValue = mapper.writeValueAsString(nodeToKafkaValue);
+        final ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(TEST_TOPIC, 0, 0L,
+                0L, TimestampType.CREATE_TIME, 0L, textToKafkaKey.length(), textToKafkaValue.length(),
+                textToKafkaKey.getBytes(UTF_8), textToKafkaValue.getBytes(UTF_8), getKafkaHeaders());
+        final ConsumerRecords<byte[], byte[]> consumerRecords = getConsumerRecords(record);
+
+        final TestRunner runner = getTestRunner(consumerRecords, TEST_TOPIC, TEST_GROUP);
+        final String keyReaderId = "key-record-reader";
+        final RecordReaderFactory keyReaderService = new JsonTreeReader();
+        runner.addControllerService(keyReaderId, keyReaderService);
+        runner.enableControllerService(keyReaderService);
+        runner.setProperty(keyReaderId, keyReaderId);
+        runner.setProperty("consume-strategy", "use-wrapper");
+        runner.setProperty("key-format", "byte-array");
+        runner.run(1);
+
+        runner.assertTransferCount("success", 1);
+        runner.assertTransferCount("parse.failure", 0);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship("success");
+        final MockFlowFile flowFile = flowFiles.iterator().next();
+        // consume strategy "use-wrapper" emits ArrayNode due to JsonRecordSetWriter
+        final JsonNode nodeFlowFile = mapper.readTree(flowFile.getContent());
+        assertTrue(nodeFlowFile instanceof ArrayNode);
+        assertEquals(1, nodeFlowFile.size());
+        // extract the NiFi json object representation of Kafka input record
+        final JsonNode flowFileValue = nodeFlowFile.iterator().next();
+        // wrapper object contains "key", "value", "headers", "metadata"
+        assertEquals(4, flowFileValue.size());
+        final JsonNode nodeWrapperKey = flowFileValue.get("key");
+        final JsonNode nodeWrapperValue = flowFileValue.get("value");
+        final JsonNode nodeWrapperHeaders = flowFileValue.get("headers");
+        final JsonNode nodeWrapperMetadata = flowFileValue.get("metadata");
+        assertNotNull(nodeWrapperKey);
+        assertNotNull(nodeWrapperValue);
+        assertNotNull(nodeWrapperHeaders);
+        assertNotNull(nodeWrapperMetadata);

Review Comment:
   A lot of the assertions in this test appear to be very repetitive. Especially this section here. Perhaps it makes sense to pull this out into a separate method, `void assertWrapperElements(JsonNode wrapper)`?



-- 
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: issues-unsubscribe@nifi.apache.org

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