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

Option to inject kafka metadata #205

Merged
merged 19 commits into from
Apr 13, 2024
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
46 changes: 33 additions & 13 deletions kafka-connect-transforms/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -49,22 +49,22 @@ It will promote the `before` or `after` element fields to top level and add the
| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` |

# JsonToMapTransform
_(Experimental)_
_(Experimental)_

The `JsonToMapTransform` SMT parses Strings as Json object payloads to infer schemas. The iceberg-kafka-connect
connector for schema-less data (e.g. the Map produced by the Kafka supplied JsonConverter) is to convert Maps into Iceberg
Structs. This is fine when the JSON is well-structured, but when you have JSON objects with dynamically
changing keys, it will lead to an explosion of columns in the Iceberg table due to schema evolutions.
The `JsonToMapTransform` SMT parses Strings as Json object payloads to infer schemas. The iceberg-kafka-connect
connector for schema-less data (e.g. the Map produced by the Kafka supplied JsonConverter) is to convert Maps into Iceberg
Structs. This is fine when the JSON is well-structured, but when you have JSON objects with dynamically
changing keys, it will lead to an explosion of columns in the Iceberg table due to schema evolutions.

This SMT is useful in situations where the JSON is not well-structured, in order to get data into Iceberg where
it can be further processed by query engines into a more manageable form. It will convert nested objects to
This SMT is useful in situations where the JSON is not well-structured, in order to get data into Iceberg where
it can be further processed by query engines into a more manageable form. It will convert nested objects to
Maps and include Map type in the Schema. The connector will respect the Schema and create Iceberg tables with Iceberg
Map (String) columns for the JSON objects.
Map (String) columns for the JSON objects.

Note:

- You must use the `stringConverter` as the `value.converter` setting for your connector, not `jsonConverter`
- It expects JSON objects (`{...}`) in those strings.
- It expects JSON objects (`{...}`) in those strings.
- Message keys, tombstones, and headers are not transformed and are passed along as-is by the SMT

## Configuration
Expand All @@ -73,16 +73,16 @@ Note:
|----------------------|------------------------------------------|
| json.root | (false) Boolean value to start at root |

The `transforms.IDENTIFIER_HERE.json.root` is meant for the most inconsistent data. It will construct a Struct with a single field
called `payload` with a Schema of `Map<String, String>`.
The `transforms.IDENTIFIER_HERE.json.root` is meant for the most inconsistent data. It will construct a Struct with a single field
called `payload` with a Schema of `Map<String, String>`.

If `transforms.IDENTIFIER_HERE.json.root` is false (the default), it will construct a Struct with inferred schemas for primitive and
array fields. Nested objects become fields of type `Map<String, String>`.

Keys with empty arrays and empty objects are filtered out from the final schema. Arrays will be typed unless the
Keys with empty arrays and empty objects are filtered out from the final schema. Arrays will be typed unless the
json arrays have mixed types in which case they are converted to arrays of strings.

Example json:
Example json:

```json
{
Expand Down Expand Up @@ -121,3 +121,23 @@ SinkRecord.value (Struct):
"array" ["1", "two", "3"]
"nested_object" Map ("some_key" : "["one", "two"]")
```

# KafkaMetadataTransform
_(Experimental)_

The `KafkaMetadata` injects `topic`, `partition`, `offset`, `timestamp` which are properties are the Kafka message.

## Configuration

| Property | Description (default value) |
|----------------|-----------------------------------------------------------------------------------|
| field_name | (_kafka_metadata) prefix for fields |
| nested | (false) if true, nests data on a struct else adds to top level as prefixed fields |
| external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) |

If `nested` is on:

`_kafka_metadata.topic`, `_kafka_metadata.partition`, `_kafka_metadata.offset`, `_kafka_metadata.timestamp`

If `nested` is off:
`_kafka_metdata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, `_kafka_metadata_timestamp`
Original file line number Diff line number Diff line change
@@ -0,0 +1,295 @@
/*
* 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 io.tabular.iceberg.connect.transforms;

import java.util.Map;
import java.util.function.Function;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.util.Requirements;
import org.apache.kafka.connect.transforms.util.SchemaUtil;
import org.apache.kafka.connect.transforms.util.SimpleConfig;

public class KafkaMetadataTransform implements Transformation<SinkRecord> {

private interface RecordAppender {

void addToSchema(SchemaBuilder builder);

void addToStruct(SinkRecord record, Struct struct);

void addToMap(SinkRecord record, Map<String, Object> map);
}

private static class NoOpRecordAppender implements RecordAppender {

@Override
public void addToSchema(SchemaBuilder builder) {}

@Override
public void addToStruct(SinkRecord record, Struct struct) {}

@Override
public void addToMap(SinkRecord record, Map<String, Object> map) {}
}

private static RecordAppender getExternalFieldAppender(
String field, Function<String, String> fieldNamer) {
if (field == null) {
return new NoOpRecordAppender();
}
String[] parts = field.split(",");
if (parts.length != 2) {
throw new ConfigException(
String.format("Could not parse %s for %s", field, EXTERNAL_KAFKA_METADATA));
}
String fieldName = fieldNamer.apply(parts[0]);
String fieldValue = parts[1];
return new RecordAppender() {

@Override
public void addToSchema(SchemaBuilder builder) {
builder.field(fieldName, Schema.STRING_SCHEMA);
}

@Override
public void addToStruct(SinkRecord record, Struct struct) {
struct.put(fieldName, fieldValue);
}

@Override
public void addToMap(SinkRecord record, Map<String, Object> map) {
map.put(fieldName, fieldValue);
}
};
}

private static final String TOPIC = "topic";
private static final String PARTITION = "partition";
private static final String OFFSET = "offset";
private static final String TIMESTAMP = "timestamp";
private static final String EXTERNAL_KAFKA_METADATA = "external_field";
private static final String KEY_METADATA_FIELD_NAME = "field_name";
private static final String KEY_METADATA_IS_NESTED = "nested";
private static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata";
private static RecordAppender recordAppender;

private static final ConfigDef CONFIG_DEF =
new ConfigDef()
.define(
KEY_METADATA_FIELD_NAME,
ConfigDef.Type.STRING,
DEFAULT_METADATA_FIELD_NAME,
ConfigDef.Importance.LOW,
"the field to append Kafka metadata under (or prefix fields with)")
.define(
KEY_METADATA_IS_NESTED,
ConfigDef.Type.BOOLEAN,
false,
ConfigDef.Importance.LOW,
"(true/false) to make a nested record under name or prefix names on the top level")
.define(
EXTERNAL_KAFKA_METADATA,
ConfigDef.Type.STRING,
null,
ConfigDef.Importance.LOW,
"key,value representing a String to be injected on Kafka metadata (e.g. Cluster)");

private static RecordAppender getRecordAppender(Map<String, ?> props) {
SimpleConfig config = new SimpleConfig(CONFIG_DEF, props);
return getRecordAppender(config);
}

private static RecordAppender getRecordAppender(SimpleConfig config) {
RecordAppender externalFieldAppender;
String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME);
Boolean nestedMetadata = config.getBoolean(KEY_METADATA_IS_NESTED);

String topicFieldName;
String partitionFieldName;
String offsetFieldName;
String timestampFieldName;

if (nestedMetadata) {
externalFieldAppender =
getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA), name -> name);

SchemaBuilder nestedSchemaBuilder = SchemaBuilder.struct();
nestedSchemaBuilder
.field(TOPIC, Schema.STRING_SCHEMA)
.field(PARTITION, Schema.INT32_SCHEMA)
.field(OFFSET, Schema.INT64_SCHEMA)
.field(TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA);
externalFieldAppender.addToSchema(nestedSchemaBuilder);

Schema nestedSchema = nestedSchemaBuilder.build();

return new RecordAppender() {
@Override
public void addToSchema(SchemaBuilder builder) {
builder.field(metadataFieldName, nestedSchema);
}

@Override
public void addToStruct(SinkRecord record, Struct struct) {
Struct nested = new Struct(nestedSchema);
nested.put(TOPIC, record.topic());
nested.put(PARTITION, record.kafkaPartition());
nested.put(OFFSET, record.kafkaOffset());
if (record.timestamp() != null) {
nested.put(TIMESTAMP, record.timestamp());
}
externalFieldAppender.addToStruct(record, nested);
struct.put(metadataFieldName, nested);
}

@Override
public void addToMap(SinkRecord record, Map<String, Object> map) {
Map<String, Object> nested = Maps.newHashMap();
nested.put(TOPIC, record.topic());
nested.put(PARTITION, record.kafkaPartition());
nested.put(OFFSET, record.kafkaOffset());
if (record.timestamp() != null) {
nested.put(TIMESTAMP, record.timestamp());
}
Comment on lines +175 to +177
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
if (record.timestamp() != null) {
nested.put(TIMESTAMP, record.timestamp());
}
nested.put(TIMESTAMP, record.timestamp());

Why can't we just do this? Do we get an error?

externalFieldAppender.addToMap(record, nested);
map.put(metadataFieldName, nested);
}
};

} else {
Function<String, String> namer = name -> String.format("%s_%s", metadataFieldName, name);
topicFieldName = namer.apply(TOPIC);
partitionFieldName = namer.apply(PARTITION);
offsetFieldName = namer.apply(OFFSET);
timestampFieldName = namer.apply(TIMESTAMP);

externalFieldAppender =
getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA), namer);
return new RecordAppender() {
@Override
public void addToSchema(SchemaBuilder builder) {
builder
.field(topicFieldName, Schema.STRING_SCHEMA)
.field(partitionFieldName, Schema.INT32_SCHEMA)
.field(offsetFieldName, Schema.INT64_SCHEMA)
.field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA);
externalFieldAppender.addToSchema(builder);
}

@Override
public void addToStruct(SinkRecord record, Struct struct) {
struct.put(topicFieldName, record.topic());
struct.put(partitionFieldName, record.kafkaPartition());
struct.put(offsetFieldName, record.kafkaOffset());
if (record.timestamp() != null) {
struct.put(timestampFieldName, record.timestamp());
}
externalFieldAppender.addToStruct(record, struct);
}

@Override
public void addToMap(SinkRecord record, Map<String, Object> map) {
map.put(topicFieldName, record.topic());
map.put(partitionFieldName, record.kafkaPartition());
map.put(offsetFieldName, record.kafkaOffset());
if (record.timestamp() != null) {
map.put(timestampFieldName, record.timestamp());
}
externalFieldAppender.addToMap(record, map);
}
};
}
}

@Override
public SinkRecord apply(SinkRecord record) {
if (record.value() == null) {
return record;
} else if (record.valueSchema() == null) {
return applySchemaless(record);
} else {
return applyWithSchema(record);
}
}

private SinkRecord applyWithSchema(SinkRecord record) {
Struct value = Requirements.requireStruct(record.value(), "KafkaMetadataTransform");
Schema newSchema = makeUpdatedSchema(record.valueSchema());
Struct newValue = new Struct(newSchema);
for (Field field : record.valueSchema().fields()) {
newValue.put(field.name(), value.get(field));
}
recordAppender.addToStruct(record, newValue);
return record.newRecord(
record.topic(),
record.kafkaPartition(),
record.keySchema(),
record.key(),
newSchema,
newValue,
record.timestamp(),
record.headers());
}

private Schema makeUpdatedSchema(Schema schema) {
SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct());
for (Field field : schema.fields()) {
builder.field(field.name(), field.schema());
}
recordAppender.addToSchema(builder);
return builder.build();
}

private SinkRecord applySchemaless(SinkRecord record) {
Map<String, Object> value = Requirements.requireMap(record.value(), "KafkaMetadata transform");
tabmatfournier marked this conversation as resolved.
Show resolved Hide resolved
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
Map<String, Object> value = Requirements.requireMap(record.value(), "KafkaMetadata transform");
Map<String, Object> value = Requirements.requireMap(record.value(), "KafkaMetadataTransform");

Map<String, Object> newValue = Maps.newHashMap(value);
recordAppender.addToMap(record, newValue);

return record.newRecord(
record.topic(),
record.kafkaPartition(),
record.keySchema(),
record.key(),
null,
newValue,
record.timestamp(),
record.headers());
}

@Override
public ConfigDef config() {
return CONFIG_DEF;
}

@Override
public void close() {}

@Override
public void configure(Map<String, ?> configs) {
recordAppender = getRecordAppender(configs);
}
}
Loading
Loading