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/08/24 17:23:29 UTC

[GitHub] [druid] lokesh-lingarajan opened a new pull request #11630: Kafka Input Format for headers, key and payload parsing.

lokesh-lingarajan opened a new pull request #11630:
URL: https://github.com/apache/druid/pull/11630


   ### Description
   
   Today we ingest a number of high cardinality metrics into Druid across dimensions such as tenant, user, TopicPartition, client ID and more. These metrics are rolled up on a per minute basis, and are very useful when looking at metrics on a partition or client basis. Events is another class of data that provides useful information about a particular incident/scenario inside a Kafka cluster. Events themselves are carried inside kafka payload, but nonetheless there are some very useful metadata that is carried in kafka headers that can serve as useful dimension for aggregation and in turn bringing better insights.
   
   PR(https://github.com/apache/druid/pull/10730) introduced support of Kafka headers in InputFormats.
   
   We still need an input format to parse out the headers and translate those into relevant columns in Druid. Until that’s implemented, none of the information available in the Kafka message headers would be exposed. So first there is a need to write an input format that can parse headers in any given format(provided we support the format) like we parse payloads today. Apart from headers there is also some useful information present in the key portion of the kafka record. We also need a way to expose the data present in the key as druid columns. We need a generic way to express at configuration time what attributes from headers, key and payload need to be ingested into druid. We need to keep the design generic enough so that users can specify different parsers for headers, key and payload.
   
   This PR is designed to solve the above by providing wrapper around any existing input formats and merging the data into a single unified Druid row.
   
   Lets look at a sample input format from the above discussion
   
   "inputFormat":
   {
       "type": "kafka",     // New input format type
       "headerLabelPrefix": "kafka.header.",   // Label prefix for header columns, this will avoid collusions while merging columns
       "recordTimestampLabelPrefix": "kafka.",  // Kafka record's timestamp is made available in case payload does not carry timestamp
       "headerFormat":  // Header parser specifying that values are of type string
       {
           "type": "string"
       },
       "valueFormat": // Value parser from json parsing
       {
           "type": "json",
           "flattenSpec": {
             "useFieldDiscovery": true,
             "fields": [...]
           }
       },
       "keyFormat":  // Key parser also from json parsing
       {
           "type": "json"
       }
   }
   
   Since we have independent sections for header, key and payload, it will enable parsing each section with its own parser, eg., headers coming in as string and payload as json. 
   
   KafkaInputFormat will be the uber class extending inputFormat interface and will be responsible for creating individual parsers for header, key and payload, blend the data resolving conflicts in columns and generating a single unified InputRow for Druid ingestion. 
   
   "headerFormat" will allow users to plug parser type for the header values and will add default header prefix as "kafka.header."(can be overridden) for attributes to avoid collision while merging attributes with payload.
   
   Kafka payload parser will be responsible for parsing the Value portion of the Kafka record. This is where most of the data will come from and we should be able to plugin existing parser. One thing to note here is that if batching is performed, then the code is augmenting header and key values to every record in the batch.
   
   Kafka key parser will handle parsing Key portion of the Kafka record and will ingest the Key with dimension name as "kafka.key".
   
   ## KafkaInputFormat Class: 
   This is the class that orchestrates sending the consumerRecord to each parser, retrieve rows, merge the columns into one final row for Druid consumption. KafkaInputformat should make sure to release the resources that gets allocated as a part of reader in CloseableIterator<InputRow> during normal and exception cases.
   
   During conflicts in dimension/metrics names, the code will prefer dimension names from payload and ignore the dimension either from headers/key. This is done so that existing input formats can be easily migrated to this new format without worrying about losing information.
   
   This code has been tested locally(mac laptop) with production data and following are the performance numbers
   Vanilla payload parser using json - 19 million records ingested/hour
   New kafka input format with header parsing - 17 million records ingested/hour
   
   Currently this code has been deployed in lab/stage environments and ingesting over 100 million records/day.
   
   @xvrl 
   
   This PR has:
   - [x ] been self-reviewed.
    - [x] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [x] added integration tests.
   - [x] been tested in a test Druid cluster.
   


-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723722456



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       This looks to be working after doing some UI wizard testing, will remove the comment.




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r720590732



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormat.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kafkainput;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.common.header.Headers;
+
+import javax.annotation.Nullable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+
+public class KafkaStringHeaderFormat implements KafkaHeaderFormat
+{
+  private final Charset encoding;
+
+  public KafkaStringHeaderFormat(
+      @JsonProperty("encoding") @Nullable String encoding

Review comment:
       did we document this anywhere?




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on pull request #11630:
URL: https://github.com/apache/druid/pull/11630#issuecomment-930308621


   @gianm - I have taken care of the review comments, could you please take another look at 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: commits-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r721601068



##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -240,11 +240,10 @@ Supported `inputFormat`s include:
 - `csv`
 - `delimited`
 - `json`
+- `kafka`
 - `avro_stream`
 - `protobuf`
 
-For more information, see [Data formats](../../ingestion/data-formats.md). You can also read [`thrift`](../extensions-contrib/thrift.md) formats using `parser`.

Review comment:
       thats a fat finger issue, will undo it.

##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,57 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |

Review comment:
       Sure will add some notes




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723690508



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       looking at the code this might work? If this works in the UI wizard, then we should remove this comment. If there are any caveats, then let's update the comment and the docs. We should still mark experimental until this is a bit more battle-tested. 




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723504537



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       I missed this, we should probably return an error here instead of returning the entire data set, or this might cause some pain in the UI wizard. We should also not this in the docs and mark it as experimental




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r720590116



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,57 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |

Review comment:
       since header values are bytes we should explain that this reads utf-8 encoded string




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r721680961



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaStringHeaderFormat.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.kafkainput;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.common.header.Headers;
+
+import javax.annotation.Nullable;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+
+public class KafkaStringHeaderFormat implements KafkaHeaderFormat
+{
+  private final Charset encoding;
+
+  public KafkaStringHeaderFormat(
+      @JsonProperty("encoding") @Nullable String encoding

Review comment:
       done

##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,57 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | [InputFormat](#input-format) | keyFormat can be any existing inputFormat to parse the kafka key. The current behavior is to only process the first entry of the input format. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | [InputFormat](#input-format) | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```
+> For any conflicts in dimension/metric names, this inputFormat will prefer kafka value's column names.
+> This will enable seemless porting of existing kafka ingestion inputFormat to this new format, with additional columns from kafka header and key.
+
+> Kafka input format fundamentally blends information from header, key and value portions of a kafka record to create a druid row. It does this by 
+> exploding individual records from the value and augmenting each of these values with the selected key/header columns.
+
+> Kafka input format also by default exposes kafka timestamp (timestampColumnName), which can be used as the primary timestamp column. 
+> One can also choose timestamp column from either key or value payload, if there is no timestamp available then the default kafka timestamp is our savior.

Review comment:
       agreed.




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723504537



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       I missed this, we should probably return an error here instead of returning the entire data set, or this might cause some pain in the UI wizard. We should also not this in the docs and mark this input format as experimental until we have fully implemented features




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723722456



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       This looks to be working after doing some UI wizard testing, will remove the comment.




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r718728004



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | Object | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```

Review comment:
       any reason why this is verbatim?




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r720590563



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,57 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | [InputFormat](#input-format) | keyFormat can be any existing inputFormat to parse the kafka key. The current behavior is to only process the first entry of the input format. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | [InputFormat](#input-format) | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```
+> For any conflicts in dimension/metric names, this inputFormat will prefer kafka value's column names.
+> This will enable seemless porting of existing kafka ingestion inputFormat to this new format, with additional columns from kafka header and key.
+
+> Kafka input format fundamentally blends information from header, key and value portions of a kafka record to create a druid row. It does this by 
+> exploding individual records from the value and augmenting each of these values with the selected key/header columns.
+
+> Kafka input format also by default exposes kafka timestamp (timestampColumnName), which can be used as the primary timestamp column. 
+> One can also choose timestamp column from either key or value payload, if there is no timestamp available then the default kafka timestamp is our savior.

Review comment:
       how does one choose the value or key timestamp? maybe an example would help.




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl merged pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl merged pull request #11630:
URL: https://github.com/apache/druid/pull/11630


   


-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r718726242



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |

Review comment:
       we should document the type to be "InputFormat" and link to the input format sections in the docs.
   
   We should also document the behavior that this will only process the first entry of the input format

##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | Object | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```

Review comment:
       any reason why this is verbatim?

##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | Object | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```
+> For any conflicts in dimension/metric names, this inputFormat will prefer kafka value's column names.
+> This will enable seemless porting of existing kafka ingestion inputFormat to this new format, with additional columns from kafka header and key.
+```
+

Review comment:
       can we explain how the input format works, and how it explodes individual records from the value and adds the corresponding key / header columns. I think that's important to understand.
   
   We also need to explain the timestamp behavior and how one can use either the Kafka timestamp or the value timestamp for when they want to migrate from the existing. We should also explain whether it is possible or not possible to use a field from the key payload as timestamp or not.




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r720589834



##########
File path: docs/development/extensions-core/kafka-ingestion.md
##########
@@ -240,11 +240,10 @@ Supported `inputFormat`s include:
 - `csv`
 - `delimited`
 - `json`
+- `kafka`
 - `avro_stream`
 - `protobuf`
 
-For more information, see [Data formats](../../ingestion/data-formats.md). You can also read [`thrift`](../extensions-contrib/thrift.md) formats using `parser`.

Review comment:
       any reason we are removing 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: commits-unsubscribe@druid.apache.org

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


[GitHub] [druid] lokesh-lingarajan commented on pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
lokesh-lingarajan commented on pull request #11630:
URL: https://github.com/apache/druid/pull/11630#issuecomment-930308621


   @gianm - I have taken care of the review comments, could you please take another look at 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: commits-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r718726242



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |

Review comment:
       we should document the type to be "InputFormat" and link to the input format sections in the docs.
   
   We should also document the behavior that this will only process the first entry of the input format




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl merged pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl merged pull request #11630:
URL: https://github.com/apache/druid/pull/11630


   


-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r723690508



##########
File path: extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.kafkainput;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaInputReader implements InputEntityReader
+{
+  private static final Logger log = new Logger(KafkaInputReader.class);
+
+  private final InputRowSchema inputRowSchema;
+  private final KafkaRecordEntity record;
+  private final KafkaHeaderReader headerParser;
+  private final InputEntityReader keyParser;
+  private final InputEntityReader valueParser;
+  private final String keyColumnName;
+  private final String timestampColumnName;
+
+  /**
+   *
+   * @param inputRowSchema Actual schema from the ingestion spec
+   * @param record kafka record containing header, key & value
+   * @param headerParser Header parser for parsing the header section, kafkaInputFormat allows users to skip header parsing section and hence an be null
+   * @param keyParser Key parser for key section, can be null as well
+   * @param valueParser Value parser is a required section in kafkaInputFormat, but because of tombstone records we can have a null parser here.
+   * @param keyColumnName Default key column name
+   * @param timestampColumnName Default kafka record's timestamp column name
+   */
+  public KafkaInputReader(
+      InputRowSchema inputRowSchema,
+      KafkaRecordEntity record,
+      KafkaHeaderReader headerParser,
+      InputEntityReader keyParser,
+      InputEntityReader valueParser,
+      String keyColumnName,
+      String timestampColumnName
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.record = record;
+    this.headerParser = headerParser;
+    this.keyParser = keyParser;
+    this.valueParser = valueParser;
+    this.keyColumnName = keyColumnName;
+    this.timestampColumnName = timestampColumnName;
+  }
+
+  private List<String> getFinalDimensionList(HashSet<String> newDimensions)
+  {
+    final List<String> schemaDimensions = inputRowSchema.getDimensionsSpec().getDimensionNames();
+    if (!schemaDimensions.isEmpty()) {
+      return schemaDimensions;
+    } else {
+      return Lists.newArrayList(
+          Sets.difference(newDimensions, inputRowSchema.getDimensionsSpec().getDimensionExclusions())
+      );
+    }
+  }
+
+  private CloseableIterator<InputRow> buildBlendedRows(InputEntityReader valueParser, Map<String, Object> headerKeyList) throws IOException
+  {
+    return valueParser.read().map(
+        r -> {
+          MapBasedInputRow valueRow;
+          try {
+            // Return type for the value parser should be of type MapBasedInputRow
+            // Parsers returning other types are not compatible currently.
+            valueRow = (MapBasedInputRow) r;
+          }
+          catch (ClassCastException e) {
+            throw new ParseException("Unsupported input format in valueFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+          }
+          Map<String, Object> event = new HashMap<>(headerKeyList);
+          /* Currently we prefer payload attributes if there is a collision in names.
+              We can change this beahvior in later changes with a config knob. This default
+              behavior lets easy porting of existing inputFormats to the new one without any changes.
+            */
+          event.putAll(valueRow.getEvent());
+
+          HashSet<String> newDimensions = new HashSet<String>(valueRow.getDimensions());
+          newDimensions.addAll(headerKeyList.keySet());
+          // Remove the dummy timestamp added in KafkaInputFormat
+          newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
+
+          return new MapBasedInputRow(
+              inputRowSchema.getTimestampSpec().extractTimestamp(event),
+              getFinalDimensionList(newDimensions),
+              event
+          );
+        }
+    );
+  }
+
+  private CloseableIterator<InputRow> buildRowsWithoutValuePayload(Map<String, Object> headerKeyList)
+  {
+    HashSet<String> newDimensions = new HashSet<String>(headerKeyList.keySet());
+    InputRow row = new MapBasedInputRow(
+        inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
+        getFinalDimensionList(newDimensions),
+        headerKeyList
+    );
+    List<InputRow> rows = Collections.singletonList(row);
+    return CloseableIterators.withEmptyBaggage(rows.iterator());
+  }
+
+  @Override
+  public CloseableIterator<InputRow> read() throws IOException
+  {
+    Map<String, Object> mergeMap = new HashMap<>();
+    if (headerParser != null) {
+      List<Pair<String, Object>> headerList = headerParser.read();
+      for (Pair<String, Object> ele : headerList) {
+        mergeMap.put(ele.lhs, ele.rhs);
+      }
+    }
+
+    // Add kafka record timestamp to the mergelist, we will skip record timestamp if the same key exists already in the header list
+    mergeMap.putIfAbsent(timestampColumnName, record.getRecord().timestamp());
+
+    if (keyParser != null) {
+      try (CloseableIterator<InputRow> keyIterator = keyParser.read()) {
+        // Key currently only takes the first row and ignores the rest.
+        if (keyIterator.hasNext()) {
+          // Return type for the key parser should be of type MapBasedInputRow
+          // Parsers returning other types are not compatible currently.
+          MapBasedInputRow keyRow = (MapBasedInputRow) keyIterator.next();
+          // Add the key to the mergeList only if the key string is not already present
+          mergeMap.putIfAbsent(keyColumnName, keyRow.getEvent().entrySet().stream().findFirst().get().getValue());
+        }
+      }
+      catch (ClassCastException e) {
+        throw new IOException("Unsupported input format in keyFormat. KafkaInputformat only supports input format that return MapBasedInputRow rows");
+      }
+    }
+
+    if (valueParser != null) {
+      return buildBlendedRows(valueParser, mergeMap);
+    } else {
+      return buildRowsWithoutValuePayload(mergeMap);
+    }
+  }
+
+  // This API is not implemented yet!

Review comment:
       looking at the code this might work? If this works in the UI wizard, then we should remove this comment. If there are any caveats, then let's update the comment and the docs. We should still mark experimental until this is a bit more battle-tested. 




-- 
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-unsubscribe@druid.apache.org

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


[GitHub] [druid] xvrl commented on a change in pull request #11630: Kafka Input Format for headers, key and payload parsing.

Posted by GitBox <gi...@apache.org>.
xvrl commented on a change in pull request #11630:
URL: https://github.com/apache/druid/pull/11630#discussion_r718729851



##########
File path: docs/ingestion/data-formats.md
##########
@@ -151,6 +151,51 @@ Be sure to change the `delimiter` to the appropriate delimiter for your data. Li
 }
 ```
 
+### KAFKA
+
+The `inputFormat` to load complete kafka record including header, key and value. An example is:
+
+```json
+"ioConfig": {
+  "inputFormat": {
+      "type": "kafka",
+      "headerLabelPrefix": "kafka.header.",
+      "timestampColumnName": "kafka.timestamp",
+      "keyColumnName": "kafka.key",
+      "headerFormat":
+      {
+        "type": "string"
+      },
+      "keyFormat":
+      {
+        "type": "json"
+      },
+      "valueFormat":
+      {
+        "type": "json"
+      }
+  },
+  ...
+}
+```
+
+The KAFKA `inputFormat` has the following components:
+
+| Field | Type | Description | Required |
+|-------|------|-------------|----------|
+| type | String | This should say `kafka`. | yes |
+| headerLabelPrefix | String | A custom label prefix for all the header columns. | no (default = "kafka.header.") |
+| timestampColumnName | String | Specifies the name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| keyColumnName | String | Specifies the name of the column for the kafka record's key.| no (default = "kafka.key") |
+| headerFormat | Object | headerFormat specifies how to parse the kafka headers. Current supported type is "string". | no |
+| keyFormat | Object | keyFormat can be any existing inputFormat to parse the kafka key. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | no |
+| valueFormat | Object | valueFormat can be any existing inputFormat to parse the kafka value payload. See [the below section](../development/extensions-core/kafka-ingestion.md#specifying-data-format) for details about specifying the input format. | yes |
+
+```
+> For any conflicts in dimension/metric names, this inputFormat will prefer kafka value's column names.
+> This will enable seemless porting of existing kafka ingestion inputFormat to this new format, with additional columns from kafka header and key.
+```
+

Review comment:
       can we explain how the input format works, and how it explodes individual records from the value and adds the corresponding key / header columns. I think that's important to understand.
   
   We also need to explain the timestamp behavior and how one can use either the Kafka timestamp or the value timestamp for when they want to migrate from the existing. We should also explain whether it is possible or not possible to use a field from the key payload as timestamp or not.




-- 
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-unsubscribe@druid.apache.org

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