diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index cc37211b..d099bee5 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -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 @@ -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`. +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`. 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`. -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 { @@ -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` diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransform.java new file mode 100644 index 00000000..9a827025 --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransform.java @@ -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 { + + private interface RecordAppender { + + void addToSchema(SchemaBuilder builder); + + void addToStruct(SinkRecord record, Struct struct); + + void addToMap(SinkRecord record, Map 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 map) {} + } + + private static RecordAppender getExternalFieldAppender( + String field, Function 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 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 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 map) { + Map 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()); + } + externalFieldAppender.addToMap(record, nested); + map.put(metadataFieldName, nested); + } + }; + + } else { + Function 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 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 value = Requirements.requireMap(record.value(), "KafkaMetadata transform"); + Map 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 configs) { + recordAppender = getRecordAppender(configs); + } +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransformTest.java new file mode 100644 index 00000000..29e1acec --- /dev/null +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransformTest.java @@ -0,0 +1,250 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.record.TimestampType; +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.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +public class KafkaMetadataTransformTest { + + private static final Schema SCHEMA = SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA); + private static final Struct VALUE_STRUCT = new Struct(SCHEMA).put("id", "value"); + private static final Map VALUE_MAP = ImmutableMap.of("id", "value"); + private static final String TOPIC = "topic"; + private static final int PARTITION = 0; + private static final long OFFSET = 1000L; + private static final long TIMESTAMP = 50000L; + private static final Schema KEY_SCHEMA = SchemaBuilder.STRING_SCHEMA; + private static final String KEY_VALUE = "key"; + + @Test + @DisplayName("should pass through null records as-is") + public void testNullRecord() { + SinkRecord record = + new SinkRecord( + TOPIC, PARTITION, null, null, null, null, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of()); + SinkRecord result = smt.apply(record); + assertThat(record).isSameAs(result); + } + } + + @Test + @DisplayName("should throw if value is not struct or map") + public void testThrowIfNotExpectedValue() { + SinkRecord recordNotMap = + new SinkRecord( + TOPIC, + PARTITION, + null, + null, + null, + "not a map", + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + SinkRecord recordNotStruct = + new SinkRecord( + TOPIC, + PARTITION, + null, + null, + SCHEMA, + "not a struct", + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of()); + assertThrows(RuntimeException.class, () -> smt.apply(recordNotMap)); + assertThrows(RuntimeException.class, () -> smt.apply(recordNotStruct)); + } + } + + @Test + @DisplayName("should append kafka metadata to structs") + public void testAppendsToStucts() { + SinkRecord record = + new SinkRecord( + TOPIC, + PARTITION, + KEY_SCHEMA, + KEY_VALUE, + SCHEMA, + VALUE_STRUCT, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of("field_name", "_some_field")); + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("id")).isEqualTo("value"); + assertThat(value.get("_some_field_topic")).isEqualTo(result.topic()); + assertThat(value.get("_some_field_partition")).isEqualTo(result.kafkaPartition()); + assertThat(value.get("_some_field_offset")).isEqualTo(result.kafkaOffset()); + assertThat(value.get("_some_field_timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } + + @Test + @DisplayName("should append kafka metadata to nested structs") + public void testAppendsToStructsNested() { + SinkRecord record = + new SinkRecord( + TOPIC, + PARTITION, + KEY_SCHEMA, + KEY_VALUE, + SCHEMA, + VALUE_STRUCT, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of("nested", "true")); + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("_kafka_metadata")).isInstanceOf(Struct.class); + Struct metadata = (Struct) value.get("_kafka_metadata"); + assertThat(metadata.get("topic")).isEqualTo(result.topic()); + assertThat(metadata.get("partition")).isEqualTo(result.kafkaPartition()); + assertThat(metadata.get("offset")).isEqualTo(result.kafkaOffset()); + assertThat(metadata.get("timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } + + @Test + @DisplayName("should append external fields to struct") + public void testAppendsToStuctsExternal() { + SinkRecord record = + new SinkRecord( + TOPIC, + PARTITION, + KEY_SCHEMA, + KEY_VALUE, + SCHEMA, + VALUE_STRUCT, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of("external_field", "external,value")); + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("id")).isEqualTo("value"); + assertThat(value.get("_kafka_metadata_topic")).isEqualTo(result.topic()); + assertThat(value.get("_kafka_metadata_partition")).isEqualTo(result.kafkaPartition()); + assertThat(value.get("_kafka_metadata_offset")).isEqualTo(result.kafkaOffset()); + assertThat(value.get("_kafka_metadata_timestamp")).isEqualTo(result.timestamp()); + assertThat(value.get("_kafka_metadata_external")).isEqualTo("value"); + } + } + + @Test + @DisplayName("throw if external field cannot be parsed") + public void testAppendsToStuctsExternalShouldThrowIfInvalid() { + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + assertThrows( + RuntimeException.class, + () -> smt.configure(ImmutableMap.of("external_field", "external,*,,,value"))); + } + } + + @Test + @DisplayName("should append kafka metadata to maps") + public void testAppendToMaps() { + SinkRecord record = + new SinkRecord( + TOPIC, + PARTITION, + null, + null, + null, + VALUE_MAP, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of()); + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + assertThat(value.get("id")).isEqualTo("value"); + assertThat(value.get("_kafka_metadata_topic")).isEqualTo(result.topic()); + assertThat(value.get("_kafka_metadata_partition")).isEqualTo(result.kafkaPartition()); + assertThat(value.get("_kafka_metadata_offset")).isEqualTo(result.kafkaOffset()); + assertThat(value.get("_kafka_metadata_timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } + + @Test + @DisplayName("should append kafka metadata to maps as nested") + public void testAppendToMapsNested() { + SinkRecord record = + new SinkRecord( + TOPIC, + PARTITION, + null, + null, + null, + VALUE_MAP, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { + smt.configure(ImmutableMap.of("nested", "true")); + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + assertThat(value.get("_kafka_metadata")).isInstanceOf(Map.class); + Map metadata = (Map) value.get("_kafka_metadata"); + assertThat(metadata.get("topic")).isEqualTo(result.topic()); + assertThat(metadata.get("partition")).isEqualTo(result.kafkaPartition()); + assertThat(metadata.get("offset")).isEqualTo(result.kafkaOffset()); + assertThat(metadata.get("timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } +}