Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

cherry pick #11630: Kafka Input Format for headers, key and payload parsing #52

Merged
merged 1 commit into from
Oct 13, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/development/extensions-core/kafka-ingestion.md
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ but unfortunately, it doesn't support all data formats supported by the legacy `
(They will be supported in the future.)

The supported `inputFormat`s include [`csv`](../../ingestion/data-formats.md#csv),
[`delimited`](../../ingestion/data-formats.md#tsv-delimited), and [`json`](../../ingestion/data-formats.md#json).
[`delimited`](../../ingestion/data-formats.md#tsv-delimited), [`kafka`](../../ingestion/data-formats.md#kafka), and [`json`](../../ingestion/data-formats.md#json).
You can also read [`avro_stream`](../../ingestion/data-formats.md#avro-stream-parser),
[`protobuf`](../../ingestion/data-formats.md#protobuf-parser),
and [`thrift`](../extensions-contrib/thrift.md) formats using `parser`.
Expand Down
69 changes: 69 additions & 0 deletions docs/ingestion/data-formats.md
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,75 @@ The `inputFormat` to load data of a delimited format. An example is:

Be sure to change the `delimiter` to the appropriate delimiter for your data. Like CSV, you must specify the columns and which subset of the columns you want indexed.

### 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:

> Note that KAFKA inputFormat is currently designated as experimental.

| 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". Since header values are bytes, the current parser by defaults reads it as UTF-8 encoded strings. There is flexibility to change this behavior by implementing your very own parser based on the encoding style. The 'encoding' type in KafkaStringHeaderFormat class needs to change with the custom implementation. | 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.
> eg.,

// Below timestampSpec chooses kafka's default timestamp that is available in kafka record
"timestampSpec":
{
"column": "kafka.timestamp",
"format": "millis"
}

// Assuming there is a timestamp field in the header and we have "kafka.header." as a desired prefix for header columns,
// below example chooses header's timestamp as a primary timestamp column
"timestampSpec":
{
"column": "kafka.header.timestamp",
"format": "millis"
}
```

### ORC

> You need to include the [`druid-orc-extensions`](../development/extensions-core/orc.md) as an extension to use the ORC input format.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.kafka.common.header.Headers;

@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "string", value = KafkaStringHeaderFormat.class)
})

public interface KafkaHeaderFormat
{
KafkaHeaderReader createReader(
Headers headers,
String headerLabelPrefix
);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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 org.apache.druid.java.util.common.Pair;

import java.util.List;

public interface KafkaHeaderReader
{
List<Pair<String, Object>> read();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,175 @@
/*
* 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 com.google.common.base.Preconditions;
import org.apache.druid.data.input.InputEntity;
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.java.util.common.DateTimes;

import javax.annotation.Nullable;
import java.io.File;
import java.util.Objects;

public class KafkaInputFormat implements InputFormat
{
private static final String DEFAULT_HEADER_COLUMN_PREFIX = "kafka.header.";
private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kafka.timestamp";
private static final String DEFAULT_KEY_COLUMN_NAME = "kafka.key";
public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp";

// Since KafkaInputFormat blends data from header, key and payload, timestamp spec can be pointing to an attribute within one of these
// 3 sections. To handle scenarios where there is no timestamp value either in key or payload, we induce an artifical timestamp value
// to avoid unnecessary parser barf out. Users in such situations can use the inputFormat's kafka record timestamp as its primary timestamp.
private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH);

private final KafkaHeaderFormat headerFormat;
private final InputFormat valueFormat;
private final InputFormat keyFormat;
private final String headerColumnPrefix;
private final String keyColumnName;
private final String timestampColumnName;

public KafkaInputFormat(
@JsonProperty("headerFormat") @Nullable KafkaHeaderFormat headerFormat,
@JsonProperty("keyFormat") @Nullable InputFormat keyFormat,
@JsonProperty("valueFormat") InputFormat valueFormat,
@JsonProperty("headerColumnPrefix") @Nullable String headerColumnPrefix,
@JsonProperty("keyColumnName") @Nullable String keyColumnName,
@JsonProperty("timestampColumnName") @Nullable String timestampColumnName
)
{
this.headerFormat = headerFormat;
this.keyFormat = keyFormat;
this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null");
this.headerColumnPrefix = headerColumnPrefix != null ? headerColumnPrefix : DEFAULT_HEADER_COLUMN_PREFIX;
this.keyColumnName = keyColumnName != null ? keyColumnName : DEFAULT_KEY_COLUMN_NAME;
this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME;
}

@Override
public boolean isSplittable()
{
return false;
}

@Override
public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
{
KafkaRecordEntity record = (KafkaRecordEntity) source;
InputRowSchema newInputRowSchema = new InputRowSchema(dummyTimestampSpec, inputRowSchema.getDimensionsSpec(), inputRowSchema.getMetricNames());
return new KafkaInputReader(
inputRowSchema,
record,
(headerFormat == null) ?
null :
headerFormat.createReader(record.getRecord().headers(), headerColumnPrefix),
(keyFormat == null || record.getRecord().key() == null) ?
null :
keyFormat.createReader(
newInputRowSchema,
new ByteEntity(record.getRecord().key()),
temporaryDirectory
),
(record.getRecord().value() == null) ?
null :
valueFormat.createReader(
newInputRowSchema,
source,
temporaryDirectory
),
keyColumnName,
timestampColumnName
);
}

@Nullable
@JsonProperty
public KafkaHeaderFormat getHeaderFormat()
{
return headerFormat;
}

@JsonProperty
public InputFormat getValueFormat()
{
return valueFormat;
}

@Nullable
@JsonProperty
public InputFormat getKeyFormat()
{
return keyFormat;
}

@Nullable
@JsonProperty
public String getHeaderColumnPrefix()
{
return headerColumnPrefix;
}

@Nullable
@JsonProperty
public String getKeyColumnName()
{
return keyColumnName;
}

@Nullable
@JsonProperty
public String getTimestampColumnName()
{
return timestampColumnName;
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KafkaInputFormat that = (KafkaInputFormat) o;
return Objects.equals(headerFormat, that.headerFormat)
&& Objects.equals(valueFormat, that.valueFormat)
&& Objects.equals(keyFormat, that.keyFormat)
&& Objects.equals(headerColumnPrefix, that.headerColumnPrefix)
&& Objects.equals(keyColumnName, that.keyColumnName)
&& Objects.equals(timestampColumnName, that.timestampColumnName);
}

@Override
public int hashCode()
{
return Objects.hash(headerFormat, valueFormat, keyFormat,
headerColumnPrefix, keyColumnName, timestampColumnName
);
}
}
Loading