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

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

                Author: ASF GitHub Bot
            Created on: 20/Apr/18 14:07
            Start Date: 20/Apr/18 14:07
    Worklog Time Spent: 10m 
      Work Description: gkumar7 commented on a change in pull request #5111: 
[BEAM-4038] Support Kafka Headers in KafkaIO
URL: https://github.com/apache/beam/pull/5111#discussion_r183061685
 
 

 ##########
 File path: 
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java
 ##########
 @@ -66,9 +76,36 @@ public void encode(KafkaRecord<K, V> value, OutputStream 
outStream) throws IOExc
         longCoder.decode(inStream),
         longCoder.decode(inStream),
         KafkaTimestampType.forOrdinal(intCoder.decode(inStream)),
+        (Headers) toHeaders(headerCoder.decode(inStream)),
         kvCoder.decode(inStream));
   }
 
+  private Object toHeaders(Iterable<KV<String, byte[]>> records) {
+    // ConsumerRecord is used to simply create a list of headers
+    ConsumerRecord<String, String> consumerRecord = new ConsumerRecord<>("", 
0, 0L, "", "");
+
+    if (!ConsumerSpEL.hasHeaders) {
+      return null;
+    } else if (!records.iterator().hasNext()) {
+      return consumerRecord.headers();
+    }
+
+    records.forEach(kv -> consumerRecord.headers().add(kv.getKey(), 
kv.getValue()));
+    return consumerRecord.headers();
+  }
+
+  private Iterable<KV<String, byte[]>> toIterable(KafkaRecord record) {
+    if (!ConsumerSpEL.hasHeaders || record.getHeaders() == null){
 
 Review comment:
   ```record.getHeaders()``` could have been null when```toHeaders()``` 
returned null even if headers were supported (in the previous change I updated 
this to return an empty iterable of headers).
   
   Agreed that it is not needed anymore.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 93219)
    Time Spent: 8.5h  (was: 8h 20m)

> Support Kafka Headers in KafkaIO
> --------------------------------
>
>                 Key: BEAM-4038
>                 URL: https://issues.apache.org/jira/browse/BEAM-4038
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-kafka
>            Reporter: Geet Kumar
>            Assignee: Raghu Angadi
>            Priority: Minor
>          Time Spent: 8.5h
>  Remaining Estimate: 0h
>
> Headers have been added to Kafka Consumer/Producer records (KAFKA-4208). The 
> purpose of this JIRA is to support this feature in KafkaIO.  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to