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/16 16:01:07 UTC

[GitHub] [nifi] greyp9 opened a new pull request, #6131: NIFI-9822 - ConsumeKafkaRecord allows writing out Kafka record key

greyp9 opened a new pull request, #6131:
URL: https://github.com/apache/nifi/pull/6131

   <!-- 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. -->
   
   # Summary
   
   [NIFI-00000](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931350038


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -239,6 +251,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
         .expressionLanguageSupported(NONE)
+        .dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)

Review Comment:
   Do you think a PublishStrategy enum would be useful here (like OutputStrategy suggested above)?



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

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r976575896


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -591,51 +607,18 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                     }
 
                     try {
+                        int recordCount = 0;
                         Record record;
                         while ((record = reader.nextRecord()) != null) {
-                            // Determine the bundle for this record.
-                            final RecordSchema recordSchema = record.getSchema();
-                            final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes, separateByKey ? consumerRecord.key() : null);
-
-                            BundleTracker tracker = bundleMap.get(bundleInfo);
-                            if (tracker == null) {
-                                FlowFile flowFile = session.create();
-                                flowFile = session.putAllAttributes(flowFile, attributes);
-
-                                final OutputStream rawOut = session.write(flowFile);
-
-                                final RecordSchema writeSchema;
-                                try {
-                                    writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
-                                } catch (final Exception e) {
-                                    logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
-
-                                    rollback(topicPartition);
-                                    yield();
-
-                                    throw new ProcessException(e);
-                                }
-
-                                writer = writerFactory.createWriter(logger, writeSchema, rawOut, flowFile);
-                                writer.beginRecordSet();
-
-                                tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer);
-                                tracker.updateFlowFile(flowFile);
-                                bundleMap.put(bundleInfo, tracker);
-                            } else {
-                                writer = tracker.recordWriter;
-                            }
-
-                            try {
-                                writer.write(record);
-                            } catch (final RuntimeException re) {
-                                handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
-                                    + "Will route message as its own FlowFile to the 'parse.failure' relationship");
-                                continue;
+                            if ((OutputStrategy.USE_WRAPPER.equals(outputStrategy)) && (consumerRecord.key() != null)) {

Review Comment:
   We don't need the `&& (consumerRecord.key() != null)` check here. We should be using the wrapper even if the key is null.



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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931354937


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -239,6 +251,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
         .expressionLanguageSupported(NONE)
+        .dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)

Review Comment:
   I'm not sure that it's necessary here. In the case of the Consumer, we are passing it down the stack (or "up the stack?") so that the lease knows what to do with it. Here, it doesn't look like we're passing the Publish Strategy to any other classes



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921359440


##########
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 understand you're suggesting a different name.  What would that be?
   
   option:
   - WRITE_VALUE, WRITE_WRAPPER
   
   Or are you saying there shouldn't be an option?



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


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

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

   Thanks for updating @greyp9 . Trying this out again.
   I just tried sending the following content via PublishKafkaRecord_2_6:
   ```
   {
     "key": {
       "type": "person"
     },
     "value": {
       "name": "Mark",
       "number": 49
     },
     "headers": {
        "headerA": "headerAValue"
     }
   }
   ```
   I set Publish Strategy to "Use Wrapper" and used a JsonTreeReader as the record reader.
   But I encountered a NullPointerException:
   ```
   2022-08-22 15:58:22,792 ERROR [Timer-Driven Process Thread-2] o.a.n.p.k.pubsub.PublishKafkaRecord_2_6 PublishKafkaRecord_2_6[id=c71d2b54-0182-1000-4979-53ab4cc8d555] Failed to send StandardFlowFileRecord[uuid=e8d03807-fa13-4620-be6a-72e014ac0838,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1661197835339-1, container=default, section=1], offset=3200, length=145],offset=0,name=e8d03807-fa13-4620-be6a-72e014ac0838,size=145] to Kafka
   java.lang.NullPointerException: null
   	at org.apache.nifi.processors.kafka.pubsub.PublisherLease.toWrapperRecord(PublisherLease.java:262)
   	at org.apache.nifi.processors.kafka.pubsub.PublisherLease.publish(PublisherLease.java:210)
   	at org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6$1.process(PublishKafkaRecord_2_6.java:521)
   	at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2693)
   	at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2661)
   	at org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_2_6.onTrigger(PublishKafkaRecord_2_6.java:513)
   	at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
   	at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1357)
   	at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:246)
   	at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:102)
   	at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
   	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
   	at java.base/java.util.concurrent.FutureTask.runAndReset(Unknown Source)
   	at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown Source)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
   	at java.base/java.lang.Thread.run(Unknown Source)
   ```
   
   One other thing that I noticed, though it's minor: When I change the "Publish Strategy" to "Use Wrapper" I'm given the option to configure the "Record Key Writer" property. But this is way down the list, about 8 or 10 properties down, so it's not at all obvious that it's made available. Probably want to move the "Record Key Writer" just after the "Publish Strategy" property sends it depends on it.


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921431303


##########
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:
   Nice!



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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921594339


##########
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:
   I think I would actually do something like:
   ```
   final Object key;
   final Object keyValue = record.getValue(messageKeyField);
   if (keyValue == null) {
     key = keyValue;
   } else if (keyValue instanceof byte[]) {
     key = (byte[]) keyValue;
   } else if (keyValue instanceof Byte[]) {
     // This case exists because in our Record API we currently don't have a BYTES type, we use an Array of type Byte, which creates a Byte[] instead of a byte[]. We should address this in the future, but we should account for the log here.
     final Byte[] bytes = (Byte[]) keyValue;
     key = new byte[bytes.length];
     for (int i=0; i < bytes.length; i++) {
       key[i] = bytes[i];
     }
   } else if (key instanceof Record) {
    ... // what you have below, to create record writer, write key, flush
   } else {
     final String keyString = keyValue.toString();
     key = keyString.getBytes(StandardCharsets.UTF_8);
   }
   ```
   I.e., just look at the object that you have. Using the schema is also feasible. But it can get really complicated if the schema indicates that it's a CHOICE type because then you'll probably still end up having to do this type of logic.



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921579179


##########
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:
   So we should examine `record.getSchema().getDataType(messageKeyField)`, rather than branch based on the `recordKeyWriterFactory`, is that right?



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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931325919


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -593,6 +609,9 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                     try {
                         Record record;
                         while ((record = reader.nextRecord()) != null) {

Review Comment:
   It's possible to have bytes that produce no record. For example, it could be JSON and contain `{}` or it could be a CSV message with just a header line and no actual data.



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

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r930324902


##########
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,133 @@ <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 (in Avro format):</p>
+<code>
+<pre>
+[
+  {
+    "type": "record",
+    "name": "kafka:ConsumeRecord:metadata",
+    "namespace": "org.apache.nifi",
+    "fields": [{
+      "name": "key",
+      "type": ["bytes", "string", "record"]
+    }, {
+      "name": "topic",
+      "type": "string"
+    }, {
+      "name": "partition",
+      "type": "int"
+    }, {
+      "name": "offset",
+      "type": "long"
+    }, {
+      "name": "timestamp",
+      "type": "long",
+      "logicalType": "timestamp-millis"
+    }]
+  },
+  {
+    "type": "record",
+    "name": "kafka:ConsumeRecord:wrapper",
+    "namespace": "org.apache.nifi",
+    "fields": [{
+      "name": "key",
+      "type": ["bytes", "string", "record"]
+    }, {
+      "name": "value",
+      "type": "record"
+    }, {
+      "name": "headers",
+      "type": "map",
+      "values": "string"
+    }, {
+      "name": "metadata",
+      "type": "kafka:ConsumeRecord:metadata"
+    }]
+  }
+]

Review Comment:
   I think I would document the schema as:
   ```
   {
     "type": "record",
     "name": "nifiRecord",
     "namespace": "org.apache.nifi",
     "fields": [{
         "name": "key",
         "type": [{
             <Schema is determined by the Key Record Reader, or will be "string" or "bytes", depending on the "Key Format" property>
           }, "null"]
       },
       {
         "name": "value",
         "type": [
           {
             <Schema is determined by the Value Record Reader, or will be "string" or "bytes", depending on the "Key Format" property>
           },
           "null"
         ]
       },
       {
         "name": "headers",
         "type": [
           { "type": "map", "values": "string", "default": {}},
           "null"]
       },
       {
         "name": "metadata",
         "type": [
           {
             "type": "record",
             "name": "metadataType",
             "fields": [
               { "name": "topic", "type": ["string", "null"] },
               { "name": "partition", "type": ["int", "null"] },
               { "name": "offset", "type": ["int", "null"] },
               { "name": "timestamp", "type": ["long", "null"] }
             ]
           },
           "null"
         ]
       }
     ]
   }
   ```



##########
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,133 @@ <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

Review Comment:
   Looks like this will likely not make it into 1.17, unfortunately. Will probably need to update that comment. Can just remove the version all together, as we generate documentation per version.



##########
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 +284,13 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .defaultValue("UTF-8")
         .required(false)
         .build();
-
+    static final PropertyDescriptor RECORD_KEY_WRITER = new PropertyDescriptor.Builder()
+            .name("record-key-writer")
+            .displayName("Record Key Writer")

Review Comment:
   Given that we're adding a Record Key Writer, we should probably change display name of "Record Writer" to "Record Value Writer" for clarity



##########
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 OUTPUT_STRATEGY = new PropertyDescriptor.Builder()
+            .name("output-strategy")
+            .displayName("Output Strategy")
+            .description("The format used to output the Kafka record into a FlowFile record.")
+            .required(true)
+            .defaultValue(OUTPUT_USE_VALUE.getValue())
+            .allowableValues(OUTPUT_USE_VALUE, OUTPUT_USE_WRAPPER)
+            .build();
+    static final PropertyDescriptor KEY_FORMAT = new PropertyDescriptor.Builder()
+            .name("key-format")
+            .displayName("Key Format")
+            .description("Specifies how to represent the Kafka Record's Key in the output")
+            .required(true)
+            .defaultValue(KEY_AS_BYTE_ARRAY.getValue())
+            .allowableValues(KEY_AS_STRING, KEY_AS_BYTE_ARRAY, KEY_AS_RECORD)
+            .dependsOn(OUTPUT_STRATEGY, OUTPUT_USE_WRAPPER)
+            .build();
+    static final PropertyDescriptor KEY_RECORD_READER = new PropertyDescriptor.Builder()

Review Comment:
   Given that we're adding a new Key Record Reader, we should change the display name of "Record Reader" to "Value Record Reader".



##########
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,77 @@ <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>'Use Content as Record Value' (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>
+
+<p>Here is an example of a Kafka Record that is emitted by <code>JsonRecordSetWriter</code> when strategy "Use Wrapper" is active:</p>

Review Comment:
   I think this section is inaccurate. This describes how the processor would behave if the "Publish Strategy" is set to "Use Content as Record Value", not "Use Wrapper".
   I do think this is as good example, as it illustrates how a key can be specified, etc. But we should also have an example of how to use the Publish Strategy of "Use Wrapper". In this case, we wouldn't specify the Key Field, and the FlowFile content would be expected to have content that matches the wrapper's format. E.g.,
   ```
   {
     "key": {"name":"Acme","number":"AC1234"},
     "value": {"address":"1234 First Street","zip":"12345","account":{"name":"Acme","number":"AC1234"}},
     "headers": {}
   }
   ```



##########
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,77 @@ <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>'Use Content as Record Value' (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>
+
+<p>Here is an example of a Kafka Record that is emitted by <code>JsonRecordSetWriter</code> when strategy "Use Wrapper" is active:</p>

Review Comment:
   It also is important here to note that the `metadata` field is currently ignored, if it is.
   However, I don't think we should actually ignore the field. Or, at least, we should optionally ignore it. It would make sense to use both the `topic` and the `partition` values of the metadata fields. This would allow us to send in data that goes to multiple topics, even as a single transaction, which would be a pretty nice feature.
   It's okay to ignore it for now, as long as the documentation very clearly & explicitly states that we ignore that field.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java:
##########
@@ -426,11 +466,13 @@ protected ConsumerPool createConsumerPool(final ProcessContext context, final Co
             }
 
             return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topics, maxUncommittedTime, securityProtocol,
-                bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume, commitOffsets);
+                    bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume,
+                    commitOffsets, outputStrategy, keyFormat, keyReaderFactory);

Review Comment:
   While it makes sense to have `String outputStrategy` at this level, as it was obtained via Property value, it can only have 2 legitimate values. As-is, the ConsumerPool has to then compare the value using `KafkaProcessorUtils.OUTPUT_USE_WRAPPER.getValue().equals(consumeStrategy)`.
   It would probably be much cleaner to instead have `enum` for the possible Output Strategies and pass that enum value into the ConsumerPool.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -593,6 +609,9 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                     try {
                         Record record;
                         while ((record = reader.nextRecord()) != null) {

Review Comment:
   There's a special case that we need to consider here.
   It's possible for a Kafka record to have a null value. In that case, it will produce no NiFi Records using the Record Reader. However, if using the Output Strategy of Wrapper, we should still output the wrapper. It may have a Key or Headers that are still of value.
   So here, we should keep track of the fact that our Record Reader has produced a Record. When we end this `while` loop, if no NiFi Record was produced, we should output a Record where the value is `null` but the Key and Headers are still populated.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -312,6 +331,8 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         properties.add(KafkaProcessorUtils.TOKEN_AUTH);
         properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
         properties.add(MESSAGE_KEY_FIELD);
+        properties.add(PUBLISH_STRATEGY);

Review Comment:
   Is probably best to add Publish Strategy before Message Key Field, since Message Key Field depends on Publish Strategy. Otherwise, we have a situation where changing the value of Publish Strategy introduce a new property above it, which is kind of an odd user experience.



##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -239,6 +251,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
         .expressionLanguageSupported(NONE)
+        .dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)

Review Comment:
   If this property now depends on Publish Strategy, it should come after Publish Strategy in the Properties list (i.e., in `this.properties`)



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931307176


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -593,6 +609,9 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                     try {
                         Record record;
                         while ((record = reader.nextRecord()) != null) {

Review Comment:
   Rather than tracking state, would it be reasonable to predetermine the decision based on `consumerRecord.value()` a few lines above?
   
   ```
   final byte[] recordBytes = consumerRecord.value() == null ? new byte[0] : consumerRecord.value();
   ```
   
   (If it is possible to emit no record even with non-null data, I guess this wouldn't work.)



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r934833581


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -593,6 +609,9 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
                     try {
                         Record record;
                         while ((record = reader.nextRecord()) != null) {

Review Comment:
   It looks like it would be necessary to extract most of the logic inside the while loop into a helper method, as it would also be needed to process this null record.  Am I missing a simpler way to do this?



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r934782331


##########
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,77 @@ <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>'Use Content as Record Value' (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>
+
+<p>Here is an example of a Kafka Record that is emitted by <code>JsonRecordSetWriter</code> when strategy "Use Wrapper" is active:</p>

Review Comment:
   Updated wrapper record to include metadata fields topic and timestamp.  If I pulled partition up to the beginning of the method, that could be included as well, but I don't know whether that would affect the application of the partition strategy.



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on PR #6131:
URL: https://github.com/apache/nifi/pull/6131#issuecomment-1185836381

   > Thanks for pushing this @greyp9 ! I think this gets us really far down field. I left some comments inline, mostly around terminology / naming conventions, and documentation. I also think the tests that were added are really helpful, but I feel like some of the code is super repetitive and some of the sets of assertions can be easily refactored into a method so that the code is a little cleaner & less verbose - unless I missed some discrepancy between very similar chunks of code that are actually different.
   
   Just pushed another commit; it should address the issues you noted.
   
   There were some efficiencies to be gained in the unit tests, but the variables needed to verify behavior made the suggested refactor difficult.  Hope the updates reach the needed bar.
   


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921429640


##########
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:
   I was able to tighten this up a bit by using `Objects.requireNonNull()`.  This highlighted a few instances where the get was subsequently unused.  But when multiple of these values are used, I couldn't think of a clear alternative. 



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931289385


##########
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,133 @@ <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 (in Avro format):</p>
+<code>
+<pre>
+[
+  {
+    "type": "record",
+    "name": "kafka:ConsumeRecord:metadata",
+    "namespace": "org.apache.nifi",
+    "fields": [{
+      "name": "key",
+      "type": ["bytes", "string", "record"]
+    }, {
+      "name": "topic",
+      "type": "string"
+    }, {
+      "name": "partition",
+      "type": "int"
+    }, {
+      "name": "offset",
+      "type": "long"
+    }, {
+      "name": "timestamp",
+      "type": "long",
+      "logicalType": "timestamp-millis"
+    }]
+  },
+  {
+    "type": "record",
+    "name": "kafka:ConsumeRecord:wrapper",
+    "namespace": "org.apache.nifi",
+    "fields": [{
+      "name": "key",
+      "type": ["bytes", "string", "record"]
+    }, {
+      "name": "value",
+      "type": "record"
+    }, {
+      "name": "headers",
+      "type": "map",
+      "values": "string"
+    }, {
+      "name": "metadata",
+      "type": "kafka:ConsumeRecord:metadata"
+    }]
+  }
+]

Review Comment:
   I had used this as a guide for expressing multiple schemas:
   https://stackoverflow.com/a/40865366
   
   Will replace with your suggestion; thanks!



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

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] markap14 merged pull request #6131: NIFI-9822 - ConsumeKafkaRecord allows writing out Kafka record key

Posted by GitBox <gi...@apache.org>.
markap14 merged PR #6131:
URL: https://github.com/apache/nifi/pull/6131


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on PR #6131:
URL: https://github.com/apache/nifi/pull/6131#issuecomment-1227755833

   > Does that make sense? Or have I confused things even worse? :)
   
   Much clearer now; thanks.  I've added a new unit test that calls out the output differences between the publisher strategies VALUE and WRAPPER, and I leveraged this to adjust the wrapper to fit the expected behavior.  (And picked up some useful jackson knowledge along the way.)  :)
   
   It should be easier to add test cases if we notice more discrepancies from the intended outputs.
   


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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r976577008


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java:
##########
@@ -653,6 +636,131 @@ private void writeRecordData(final ProcessSession session, final List<ConsumerRe
         }
     }
 
+    private RecordSetWriter writeRecord(final ProcessSession session, final ConsumerRecord<byte[], byte[]> consumerRecord, final TopicPartition topicPartition,
+                                        final Record record, final Map<String, String> attributes) throws SchemaNotFoundException, IOException {
+            RecordSetWriter writer = null;
+            // Determine the bundle for this record.
+            final RecordSchema recordSchema = record.getSchema();
+            final BundleInformation bundleInfo = new BundleInformation(topicPartition, recordSchema, attributes, separateByKey ? consumerRecord.key() : null);
+
+            BundleTracker tracker = bundleMap.get(bundleInfo);
+            if (tracker == null) {
+                FlowFile flowFile = session.create();
+                flowFile = session.putAllAttributes(flowFile, attributes);
+
+                final OutputStream rawOut = session.write(flowFile);
+
+                final RecordSchema writeSchema;
+                try {
+                    writeSchema = writerFactory.getSchema(flowFile.getAttributes(), recordSchema);
+                } catch (final Exception e) {
+                    logger.error("Failed to obtain Schema for FlowFile. Will roll back the Kafka message offsets.", e);
+
+                    rollback(topicPartition);
+                    yield();
+
+                    throw new ProcessException(e);
+                }
+
+                writer = writerFactory.createWriter(logger, writeSchema, rawOut, flowFile);
+                writer.beginRecordSet();
+
+                tracker = new BundleTracker(consumerRecord, topicPartition, keyEncoding, writer);
+                tracker.updateFlowFile(flowFile);
+                bundleMap.put(bundleInfo, tracker);
+            } else {
+                writer = tracker.recordWriter;
+            }
+
+            try {
+                writer.write(record);
+            } catch (final RuntimeException re) {
+                handleParseFailure(consumerRecord, session, re, "Failed to write message from Kafka using the configured Record Writer. "
+                        + "Will route message as its own FlowFile to the 'parse.failure' relationship");
+                return writer;
+            }
+
+            tracker.incrementRecordCount(1L, consumerRecord.offset(), consumerRecord.leaderEpoch().orElse(null));
+            session.adjustCounter("Records Received", 1L, false);
+        return writer;
+    }
+
+    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));

Review Comment:
   If the original key was null, we should not be using `new String(key, StandardCharsets.UTF_8)` because that generates an empty string. Instead, we should just use `null`.



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


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

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

   @greyp9 So, given the above (i will address in reverse order because I think you started with the hardest to describe and got easier as you went down the list) :) ...
   
   The last point, about "Message Key Field" property I think is perfectly accurate.
   
   As for the headers:
   - I think what you said is accurate, but to clarify:
   - If (Use Wrapper) - the headers to send would be a single header. Its name would be "headerA" and its value would be "headerAValue". FlowFile attributes would not be sent as headers.
   - Else, the headers would be any FlowFile attribute that matches the "Attributes to Send as Headers (Regex)" property
   
   Now, as for the other...
   ```
   if (Use Wrapper) {
       Kafka Record:
       Key = { "type": "person" }
       Value = { "name": "Mark", "number": 49 }
       Headers = A single header with name "headerA", value "headerAValue"
   } else {
       Kafka Record:
       Key = <Depends on the value of the 'Message Key Field' property>
       Value = <The entire JSON payload. I.e.,:>
       {
                          "key": {
                              "type": "person"
                           },
                          "value": {
                               "name": "Mark",
                               "number": 49
                          },
                          "headers": {
                                "headerA": "headerAValue"
                          }
        }
       Headers = <Whatever matches the 'Attributes to Send as Headers (Regex)' property>
   }
   ```
   
   So, in short, if Use Wrapper, the incoming FlowFile must have 3 fields:
   Key. This becomes the kafka message key.
   Value. This becomes the contents of the kafka message.
   Headers. This becomes the headers attached to the kafka message.
   Any other fields, such as metadata, would be ignored.
   
   If NOT using wrapper, it would function as it always has. The entire contents of the Record go as the kafka message payload. The key and headers are determined based on the configured "Message Key Field" and "Attributes to Send as Headers (Regex)" properties.
   
   Does that make sense? Or have I confused things even worse? :)


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


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

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r991543917


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java:
##########
@@ -188,18 +202,30 @@ void publish(final FlowFile flowFile, final RecordSet recordSet, final RecordSet
                 baos.reset();
 
                 Map<String, String> additionalAttributes = Collections.emptyMap();
-                try (final RecordSetWriter writer = writerFactory.createWriter(logger, schema, baos, flowFile)) {
-                    final WriteResult writeResult = writer.write(record);
-                    additionalAttributes = writeResult.getAttributes();
-                    writer.flush();
+                final List<Header> headers;
+                final byte[] messageContent;
+                final byte[] messageKey;
+                if (PublishStrategy.USE_WRAPPER.equals(publishStrategy)) {
+                    headers = toHeadersWrapper(record.getValue("headers"));
+                    final Object key = record.getValue("key");
+                    final Object value = record.getValue("value");
+                    logger.trace("Key: {}", key);
+                    logger.trace("Value: {}", value);

Review Comment:
   Recommend removing these trace logs from the final version.



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


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

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r921422679


##########
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:
   Yes, sorry, I was suggesting renaming it. Perhaps something like "Use Content as Record Value" or "FLowFile Content as Record Value" or something like that? Was just trying to convey that we aren't *only* writing the record value, a key and headers may also be written.



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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6131:
URL: https://github.com/apache/nifi/pull/6131#discussion_r931631101


##########
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java:
##########
@@ -239,6 +251,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             + "If not specified, no FlowFile attributes will be added as headers.")
         .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
         .expressionLanguageSupported(NONE)
+        .dependsOn(PUBLISH_STRATEGY, PUBLISH_USE_VALUE)

Review Comment:
   Oof.  That's an oversight.  The propagation from processor to pool to lease is needed, so I'll do enum here as well.



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


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

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

   I also tried changing the "Publish Strategy" to "Use Record Content as Value" so that I could set the "Key Field." I then set the Key Field to `key`. And then changed the Publish Strategy back to "Use Wrapper".
   This time, because the Key Field was specified, it sent the data. But what it sent was not correct.
   It sent the key correctly. But for the Kafka message, it sent the entire payload, not just the `value` section.
   And it sent no headers. So it appears to behave as if the Publish Strategy still was set to "Use Record Content as Value"


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on PR #6131:
URL: https://github.com/apache/nifi/pull/6131#issuecomment-1224612052

   > Thanks for updating @greyp9 . Trying this out again. I just tried sending the following content via PublishKafkaRecord_2_6:
   
   > I set Publish Strategy to "Use Wrapper" and used a JsonTreeReader as the record reader. But I encountered a NullPointerException:
   > 
   
   Updated to validate content of message key.
   
   
   > One other thing that I noticed, though it's minor: When I change the "Publish Strategy" to "Use Wrapper" I'm given the option to configure the "Record Key Writer" property. But this is way down the list, about 8 or 10 properties down, so it's not at all obvious that it's made available. Probably want to move the "Record Key Writer" just after the "Publish Strategy" property sends it depends on it.
   
   Moved to just after "Publish Strategy".


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


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

Posted by GitBox <gi...@apache.org>.
greyp9 commented on PR #6131:
URL: https://github.com/apache/nifi/pull/6131#issuecomment-1224730836

   > ```
   > {
   >   "key": {
   >     "type": "person"
   >   },
   >   "value": {
   >     "name": "Mark",
   >     "number": 49
   >   },
   >   "headers": {
   >      "headerA": "headerAValue"
   >   }
   > }
   > ```
   
   Thanks.  There are enough permutations related to these changes such that I'm not sure what should be emitted for all of them.  I've made some simplifying assumptions; those may need adjustments.
   
   Maybe a concrete example would help.  Given the record above, and the processor PublishKafkaRecord_2_6:
   
   [Processor Property] PublishStrategy
   ```
   if (Use Wrapper) {
       Kafka Record:
       Key = ?
       Value = ?
       Headers = ?
   } else {
       Kafka Record:
       Key = ?
       Value = ?
       Headers = ?
   }
   ```
   
   ---
   
   [Processor Property] Attributes To Send as Headers (Regex)
   ```
   if (Use Wrapper) {
       send attributes in wrapper record (do not add to kafka headers)
   } else {
       send attributes in kafka headers
   }
   ```
   is this high-level logic valid?
   
   ---
   
   [Processor Property] Message Key Field
   ```
   if (Use Wrapper) {
       ignore (null key)
   } else {
       send key (if record field exists) in Kafka key
   }
   ```
   
   is this right?
   


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