npawar commented on code in PR #9224:
URL: https://github.com/apache/pinot/pull/9224#discussion_r976903107


##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/RowMetadataExtractor.java:
##########
@@ -18,15 +18,35 @@
  */
 package org.apache.pinot.plugin.stream.kafka20;
 
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.pinot.spi.data.readers.GenericRow;
 import org.apache.pinot.spi.stream.RowMetadata;
 import org.apache.pinot.spi.stream.StreamMessageMetadata;
 
 
 @FunctionalInterface
 public interface RowMetadataExtractor {
   static RowMetadataExtractor build(boolean populateMetadata) {
-    return populateMetadata ? record -> new 
StreamMessageMetadata(record.timestamp()) : record -> null;
+    return record -> {
+      if (!populateMetadata) {
+        return null;
+      }
+      GenericRow headerGenericRow = new GenericRow();
+      Headers headers = record.headers();
+      if (headers != null) {
+        Header[] headersArray = headers.toArray();
+        for (Header header : headersArray) {
+          headerGenericRow.putValue(header.key(), header.value());
+        }
+      }
+      Map<String, String> metadata = new HashMap<>();
+      metadata.put("offset", String.valueOf(record.offset()));

Review Comment:
   since we're putting the "offset" into the StreamMessageMetadata in this 
kafka specific rowMetadataExtractor, do we still need the "offset" as param in 
KafkaStreamMessage? the calls can extract it from the StreamMessageMetadata 
object instead? just trying to remove that special casing, since it seems we 
already are encapsulating it inside the StreamMessageMetadata 



##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/RowMetadataExtractor.java:
##########
@@ -18,15 +18,35 @@
  */
 package org.apache.pinot.plugin.stream.kafka20;
 
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.pinot.spi.data.readers.GenericRow;
 import org.apache.pinot.spi.stream.RowMetadata;
 import org.apache.pinot.spi.stream.StreamMessageMetadata;
 
 
 @FunctionalInterface
 public interface RowMetadataExtractor {
   static RowMetadataExtractor build(boolean populateMetadata) {
-    return populateMetadata ? record -> new 
StreamMessageMetadata(record.timestamp()) : record -> null;
+    return record -> {
+      if (!populateMetadata) {
+        return null;
+      }
+      GenericRow headerGenericRow = new GenericRow();
+      Headers headers = record.headers();
+      if (headers != null) {
+        Header[] headersArray = headers.toArray();
+        for (Header header : headersArray) {
+          headerGenericRow.putValue(header.key(), header.value());
+        }
+      }
+      Map<String, String> metadata = new HashMap<>();
+      metadata.put("offset", String.valueOf(record.offset()));

Review Comment:
   just realized that this RowMetadataExtractor is within the Kafka module. So 
to support metadata extraction for other streams, we'd have to add respective 
implementations? in which case (not introduced by this PR but) should this 
class be renamed to KafkaRowMetadataExtractor?



##########
pinot-spi/src/main/java/org/apache/pinot/spi/stream/RowMetadata.java:
##########
@@ -31,6 +32,8 @@
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public interface RowMetadata {
+  GenericRow EMPTY_ROW = new GenericRow();

Review Comment:
   and StreamMessageMetadata would still extend RowMetadata or you'd completely 
separate and unlink the 2? 
   If it's hard to remove, i'm okay with leaving this as is for now



##########
pinot-plugins/pinot-stream-ingestion/pinot-kafka-2.0/src/main/java/org/apache/pinot/plugin/stream/kafka20/KafkaStreamMessage.java:
##########
@@ -0,0 +1,51 @@
+/**
+ * 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.pinot.plugin.stream.kafka20;
+
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.stream.StreamMessage;
+import org.apache.pinot.spi.stream.StreamMessageMetadata;
+
+
+public class KafkaStreamMessage extends StreamMessage {
+  private static final GenericRow EMPTY_ROW_REUSE = new GenericRow();
+
+  // should distinguish stream-specific record metadata in the table??
+  private final long _offset;

Review Comment:
   I think the latter would be clean - "create a KafkaStreamMessageMetadata 
that extends StreamMessageMetadata".  In which case we wouldn't need a impl of 
StreamMessage for every stream, we'd just need to add an impl of 
StreamMessageMetadata for each stream. wdyt?



-- 
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: commits-unsubscr...@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to