[ 
https://issues.apache.org/jira/browse/BEAM-12008?focusedWorklogId=621147&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-621147
 ]

ASF GitHub Bot logged work on BEAM-12008:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/Jul/21 19:29
            Start Date: 09/Jul/21 19:29
    Worklog Time Spent: 10m 
      Work Description: boyuanzz commented on a change in pull request #15090:
URL: https://github.com/apache/beam/pull/15090#discussion_r667122540



##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -847,6 +858,18 @@ public void setTimestampPolicy(String timestampPolicy) {
       }
     }
 
+    /**
+     * Update SupportsNullKeys for present of null keys
+     *
+     * <p>By default, withSupportsNullKeys is {@code false} and will invoke 
{@link KafkaRecordCoder}
+     * as normal. In this case, {@link KafkaRecordCoder} will not be able to 
handle null keys.
+     * When nullKeyFlag is {@code true}, it wraps the key coder with a {@link 
NullableCoder} before
+     * invoking {@link KafkaRecordCoder}. In this case, it can handle null 
keys.
+     */
+    public Read<K, V> withSupportsNullKeys() {

Review comment:
       `withNullabeKeys`?

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -847,6 +858,18 @@ public void setTimestampPolicy(String timestampPolicy) {
       }
     }
 
+    /**
+     * Update SupportsNullKeys for present of null keys

Review comment:
       ```suggestion
        * Indicates whether the key of {@link KafkaRecord} could be null.
   ```

##########
File path: 
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/NullableKeyKafkaRecordCoderTest.java
##########
@@ -0,0 +1,76 @@
+/*

Review comment:
       Please remove this file if it's no longer needed.

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -768,6 +778,8 @@ private static Coder resolveCoder(Class deserializer) {
         }
         throw new RuntimeException("Couldn't resolve coder for Deserializer: " 
+ deserializer);
       }
+
+

Review comment:
       Please remove additional whitespace changes.

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/NullableKeyKafkaRecordCoder.java
##########
@@ -0,0 +1,160 @@
+/*

Review comment:
       Please remove this file if it's not needed anymore.

##########
File path: 
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java
##########
@@ -258,6 +301,17 @@ private void cancelIfTimeouted(PipelineResult readResult, 
PipelineResult.State r
         .withTopic(options.getKafkaTopic());
   }
 
+  private KafkaIO.Read<byte[], String> readFromKafkaNullKey() {
+    return KafkaIO.<byte[], String>read()
+        .withSupportsNullKeys()
+        .withBootstrapServers(options.getKafkaBootstrapServerAddresses())
+        .withConsumerConfigUpdates(ImmutableMap.of("auto.offset.reset", 
"earliest"))
+        .withTopic(options.getKafkaTopic())
+        .withMaxNumRecords(100)

Review comment:
       Usually hardcoding a number in a util function is not perferred. If your 
test is the only place using this, it might be better to have the test 
construct Kafka read directly.

##########
File path: 
sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java
##########
@@ -166,6 +174,41 @@ public void 
testKafkaIOReadsAndWritesCorrectlyInStreaming() throws IOException {
     }
   }
 
+  @Test
+  public void testKafkaIOReadsAndWritesCorrectlyInBatchNullKey() throws 
IOException {
+    List<String> values = new ArrayList<>();
+    for (int i = 0; i < 100; i++) {
+      values.add("value" + Integer.toString(i));
+    }
+    PCollection<String> writeInput =
+        writePipeline.apply(Create.of(values)).setCoder(StringUtf8Coder.of());
+
+    writeInput.apply(
+        KafkaIO.<byte[], String>write()
+            .withBootstrapServers(options.getKafkaBootstrapServerAddresses())
+            .withTopic(options.getKafkaTopic())
+            .withValueSerializer(StringSerializer.class)
+            .values());
+
+    PCollection<String> readOutput =
+        readPipeline
+            .apply("Read from bounded Kafka", readFromKafkaNullKey())
+            .apply("Materialize input", Reshuffle.viaRandomKey())
+            .apply(
+                "Map records to strings", MapElements.via(new 
MapKafkaRecordsToStringsNullKey()));

Review comment:
       You can use a lambda here, like: 
https://github.com/apache/beam/blob/243128a8fc52798e1b58b0cf1a271d95ee7aa241/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/PayloadSerializerKafkaTable.java#L53-L54

##########
File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -847,6 +858,18 @@ public void setTimestampPolicy(String timestampPolicy) {
       }
     }
 
+    /**
+     * Update SupportsNullKeys for present of null keys
+     *
+     * <p>By default, withSupportsNullKeys is {@code false} and will invoke 
{@link KafkaRecordCoder}
+     * as normal. In this case, {@link KafkaRecordCoder} will not be able to 
handle null keys.
+     * When nullKeyFlag is {@code true}, it wraps the key coder with a {@link 
NullableCoder} before
+     * invoking {@link KafkaRecordCoder}. In this case, it can handle null 
keys.
+     */

Review comment:
       ```suggestion
        * <p>By specifying {@link withNullableKeys}, {@link KafkaIO.Read} is 
able to handle {@link KafkaRecord} with nullable keys. Otherwise, {@link 
KafkaIO.Read} will assume the key from {@link KafkaRecord} is not null all the 
time. Reading {@link KafkaRecord} with nullable keys but without specifying 
{@link withNullableKeys} may result in pipeline failures.
        */
   ```




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

To unsubscribe, e-mail: [email protected]

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 621147)
    Time Spent: 6.5h  (was: 6h 20m)

> KafkaIO does not handle null keys
> ---------------------------------
>
>                 Key: BEAM-12008
>                 URL: https://issues.apache.org/jira/browse/BEAM-12008
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-kafka
>            Reporter: Daniel Collins
>            Assignee: Weiwen Xu
>            Priority: P2
>              Labels: stale-P2
>          Time Spent: 6.5h
>  Remaining Estimate: 0h
>
> Kafka 
> [ConsumerRecord|https://kafka.apache.org/27/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html#key--]
>  and 
> [ProducerRecord|https://kafka.apache.org/27/javadoc/org/apache/kafka/clients/producer/ProducerRecord.html#key--]
>  'key' fields are explicitly allowed to be null. In addition, on the producer 
> side, setting a null key is the way that the user indicates that they want a 
> [random partition for their 
> message|[https://github.com/apache/kafka/blob/9adfac280392da0837cfd8d582bc540951e94087/clients/src/main/java/org/apache/kafka/clients/producer/internals/DefaultPartitioner.java#L67].]
>  
> Beam KafkaIO does not support null keys in byte[] mode (read side: 
> [https://github.com/apache/beam/blob/9e0997760cf3320f1a1d0c4342d3dff559a25775/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L727|https://github.com/apache/beam/blob/9e0997760cf3320f1a1d0c4342d3dff559a25775/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L727)]
> write side: 
> [https://github.com/apache/beam/blob/9e0997760cf3320f1a1d0c4342d3dff559a25775/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java#L58])
>  
> since it would defer to ByteArrayCoder which does not support null arrays.
>  
> BeamKafkaTable suffers the same issue 
> https://github.com/apache/beam/blob/9e0997760cf3320f1a1d0c4342d3dff559a25775/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java#L144



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to