You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/01/07 02:22:12 UTC

[GitHub] [druid] himanshug commented on a change in pull request #10730: Introduce KafkaRecordEntity to support Kafka headers in InputFormats

himanshug commented on a change in pull request #10730:
URL: https://github.com/apache/druid/pull/10730#discussion_r553063732



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
##########
@@ -119,15 +120,16 @@ public void seekToLatest(Set<StreamPartition<Integer>> partitions)
 
   @Nonnull
   @Override
-  public List<OrderedPartitionableRecord<Integer, Long>> poll(long timeout)
+  public List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> poll(long timeout)
   {
-    List<OrderedPartitionableRecord<Integer, Long>> polledRecords = new ArrayList<>();
+    List<OrderedPartitionableRecord<Integer, Long, KafkaRecordEntity>> polledRecords = new ArrayList<>();
     for (ConsumerRecord<byte[], byte[]> record : consumer.poll(Duration.ofMillis(timeout))) {
+
       polledRecords.add(new OrderedPartitionableRecord<>(
           record.topic(),
           record.partition(),
           record.offset(),
-          record.value() == null ? null : ImmutableList.of(record.value())
+          ImmutableList.of(new KafkaRecordEntity(record))

Review comment:
       I think it is fine as long as it doesn't cause an unhandled NPE somewhere downstream and just counts as `unparseable` record for existing pipelines.

##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.druid.data.input.kafka;
+
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+
+public class KafkaRecordEntity extends ByteEntity
+{
+  private final ConsumerRecord<byte[], byte[]> record;
+
+  public KafkaRecordEntity(ConsumerRecord<byte[], byte[]> record)
+  {
+    super(record.value());
+    this.record = record;
+  }
+
+  public ConsumerRecord<byte[], byte[]> getRecord()

Review comment:
       not used anywhere , is this only to allow writing extensions with custom InputFormat which can take advantage of the extra metadata available ?

##########
File path: indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java
##########
@@ -55,7 +54,7 @@ public OrderedPartitionableRecord(
     this.stream = stream;
     this.partitionId = partitionId;
     this.sequenceNumber = sequenceNumber;
-    this.data = data == null ? ImmutableList.of() : data;
+    this.data = data;

Review comment:
       why remove the null check and conversion to empty list ?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org