From 7d88b1d73e8c5d9d8163bd3be4db812ef093ba6a Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 16:07:22 -0800 Subject: [PATCH 01/17] option-to-inject-kafka-metadata - add option to inject kafka metadata if Debezium or DMS messages are SinkRecords --- .../connect/transforms/DebeziumTransform.java | 43 +++++- .../connect/transforms/DmsTransform.java | 43 +++++- .../util/KafkaMetadataAppender.java | 136 +++++++++++++++++ .../transforms/DebeziumTransformTest.java | 57 ++++++- .../connect/transforms/DmsTransformTest.java | 21 +++ .../util/KafkaMetadataAppenderTest.java | 142 ++++++++++++++++++ 6 files changed, 429 insertions(+), 13 deletions(-) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java create mode 100644 kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 1e79e211..6ce9401f 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -19,6 +19,9 @@ package io.tabular.iceberg.connect.transforms; import java.util.Map; +import java.util.Objects; + +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -43,6 +46,7 @@ public class DebeziumTransform> implements Transforma private static final String CDC_TARGET_PATTERN = "cdc.target.pattern"; private static final String DB_PLACEHOLDER = "{db}"; private static final String TABLE_PLACEHOLDER = "{table}"; + private static KafkaMetadataAppender kafkaAppender = null; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -51,7 +55,24 @@ public class DebeziumTransform> implements Transforma ConfigDef.Type.STRING, null, Importance.MEDIUM, - "Pattern to use for setting the CDC target field value."); + "Pattern to use for setting the CDC target field value.") + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + Importance.LOW, + "Include appending of Kafka metadata to SinkRecord" + ) + .define(KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + Importance.LOW, + "field to append Kafka metadata under") + .define(KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); private String cdcTargetPattern; @@ -59,6 +80,9 @@ public class DebeziumTransform> implements Transforma public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); cdcTargetPattern = config.getString(CDC_TARGET_PATTERN); + if (config.getBoolean(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)) { + kafkaAppender = KafkaMetadataAppender.from(config); + } } @Override @@ -110,6 +134,12 @@ private R applyWithSchema(R record) { } newValue.put(CdcConstants.COL_CDC, cdcMetadata); + if (Objects.nonNull(kafkaAppender)) { + if (record instanceof SinkRecord) { + kafkaAppender.appendToStruct((SinkRecord) record, newValue); + } + } + return record.newRecord( record.topic(), record.kafkaPartition(), @@ -138,12 +168,18 @@ private R applySchemaless(R record) { return null; } + // create the new value + Map newValue = Maps.newHashMap((Map) payload); + // create the CDC metadata Map cdcMetadata = Maps.newHashMap(); cdcMetadata.put(CdcConstants.COL_OP, op); cdcMetadata.put(CdcConstants.COL_TS, value.get("ts_ms")); if (record instanceof SinkRecord) { cdcMetadata.put(CdcConstants.COL_OFFSET, ((SinkRecord) record).kafkaOffset()); + if (Objects.nonNull(kafkaAppender)) { + kafkaAppender.appendToMap((SinkRecord) record, newValue); + } } setTableAndTargetFromSourceMap(value.get("source"), cdcMetadata); @@ -151,8 +187,6 @@ private R applySchemaless(R record) { cdcMetadata.put(CdcConstants.COL_KEY, record.key()); } - // create the new value - Map newValue = Maps.newHashMap((Map) payload); newValue.put(CdcConstants.COL_CDC, cdcMetadata); return record.newRecord( @@ -238,6 +272,9 @@ private Schema makeUpdatedSchema(Schema schema, Schema cdcSchema) { builder.field(CdcConstants.COL_CDC, cdcSchema); + if (Objects.nonNull(kafkaAppender)) { + kafkaAppender.appendSchema(builder); + } return builder.build(); } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index 4db4bb3a..e50b8545 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -19,18 +19,44 @@ package io.tabular.iceberg.connect.transforms; import java.util.Map; +import java.util.Objects; + +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; +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.SimpleConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DmsTransform> implements Transformation { private static final Logger LOG = LoggerFactory.getLogger(DmsTransform.class.getName()); - private static final ConfigDef EMPTY_CONFIG = new ConfigDef(); + public static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord" + ) + .define(KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define(KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + + private static KafkaMetadataAppender kafkaAppender = null; + @Override public R apply(R record) { @@ -82,6 +108,12 @@ private R applySchemaless(R record) { Map newValue = Maps.newHashMap((Map) dataObj); newValue.put(CdcConstants.COL_CDC, cdcMetadata); + if(Objects.nonNull(kafkaAppender)) { + if (record instanceof SinkRecord) { + kafkaAppender.appendToMap((SinkRecord) record, newValue); + } + } + return record.newRecord( record.topic(), record.kafkaPartition(), @@ -94,12 +126,17 @@ private R applySchemaless(R record) { @Override public ConfigDef config() { - return EMPTY_CONFIG; + return CONFIG_DEF; } @Override public void close() {} @Override - public void configure(Map configs) {} + public void configure(Map configs) { + SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + if (config.getBoolean(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)) { + kafkaAppender = KafkaMetadataAppender.from(config); + } + } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java new file mode 100644 index 00000000..f7cda1cd --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -0,0 +1,136 @@ +package io.tabular.iceberg.connect.transforms.util; + +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.config.ConfigException; +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.util.SimpleConfig; + +import java.util.Map; +import java.util.Objects; + +public class KafkaMetadataAppender { + + public static final String INCLUDE_KAFKA_METADATA = "cdc.kafka.include_metadata"; + public static final String EXTERNAL_KAFKA_METADATA = "cdc.kafka.external_field"; + public static final String KEY_METADATA_FIELD_NAME = "cdc.kafka.metadata_field"; + public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; + + private final ExternalKafkaData externalData; + + private final Schema schema; + + private final String metadataFieldName; + + public KafkaMetadataAppender(ExternalKafkaData externalData, Schema schema, String metadataFieldName) { + this.externalData = externalData; + this.schema = schema; + this.metadataFieldName = metadataFieldName; + } + + public interface ExternalKafkaData { + SchemaBuilder addToSchema(SchemaBuilder builder); + Struct addToStruct(Struct struct); + Map addToMap(Map map); + } + public static class ExternalStringKafkaData implements ExternalKafkaData{ + private final String name; + private final String value; + + public ExternalStringKafkaData(String name, String value) { + this.name = name; + this.value = value; + } + + public static ExternalKafkaData parse(String field) { + if(field.equals("none")) { + return new EmptyExternalData(); + } + String[] parts = field.split(","); + if (parts.length != 2) { + throw new ConfigException(String.format("Could not parse %s for %s", field, KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA)); + } + return new ExternalStringKafkaData(parts[0], parts[1]); + } + + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(this.name, Schema.STRING_SCHEMA); + } + + @Override + public Struct addToStruct(Struct struct) { + return struct.put(this.name, this.value); + } + + @Override + public Map addToMap(Map map) { + map.put(this.name, this.value); + return map; + } + } + + public static class EmptyExternalData implements ExternalKafkaData { + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder; + } + + @Override + public Struct addToStruct(Struct struct) { + return struct; + } + + @Override + public Map addToMap(Map map) { + return map; + } + } + + public static KafkaMetadataAppender from(SimpleConfig config) { + ExternalKafkaData externalAppender = ExternalStringKafkaData.parse(config.getString(EXTERNAL_KAFKA_METADATA)); + String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME); + + SchemaBuilder schema = SchemaBuilder.struct(); + + externalAppender.addToSchema(schema) + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + // TODO headers + return new KafkaMetadataAppender(externalAppender, schema.build(), metadataFieldName); + } + + public SchemaBuilder appendSchema(SchemaBuilder schema) { + return schema.field(this.metadataFieldName, this.schema); + } + + public Struct appendToStruct(SinkRecord record, Struct struct) { + Struct metadata = new Struct(this.schema); + externalData.addToStruct(metadata); + metadata.put("topic", record.topic()); + metadata.put("partition", record.kafkaPartition()); + metadata.put("offset", record.kafkaOffset()); + if (Objects.nonNull(record.timestamp())) { + metadata.put("timestamp", record.timestamp()); + } + struct.put(this.metadataFieldName, metadata); + return struct; + } + + public Map appendToMap(SinkRecord record, Map map) { + Map metadata = Maps.newHashMap(); + externalData.addToMap(metadata); + metadata.put("topic", record.topic()); + metadata.put("partition", record.kafkaPartition()); + metadata.put("offset", record.kafkaOffset()); + if (Objects.nonNull(record.timestamp())) { + metadata.put("timestamp", record.timestamp()); + } + map.put(this.metadataFieldName, metadata); + return map; + } +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index c9f34848..e0410dbe 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -23,6 +23,8 @@ import java.math.BigDecimal; import java.time.Instant; import java.util.Map; + +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; @@ -89,6 +91,27 @@ public void testDebeziumTransformSchemaless() { assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); assertThat(cdcMetadata.get("key")).isInstanceOf(Map.class); + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testDebeziumTransformSchemalessAndKafkaMetadata() { + try (DebeziumTransform smt = new DebeziumTransform<>()) { + smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + + Map event = createDebeziumEventMap("u"); + Map key = ImmutableMap.of("account_id", 1L); + SinkRecord record = new SinkRecord("topic", 0, null, key, null, event, 0); + + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); + Map metadata = (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + assertThat(metadata.get("topic")).isEqualTo("topic"); } } @@ -97,6 +120,30 @@ public void testDebeziumTransformWithSchema() { try (DebeziumTransform smt = new DebeziumTransform<>()) { smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x")); + Map event = createDebeziumEventMap("u"); + Map key = ImmutableMap.of("account_id", 1L); + SinkRecord record = new SinkRecord("topic", 0, null, key, null, event, 0); + + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + + assertThat(value.get("account_id")).isEqualTo(1); + + Map cdcMetadata = (Map) value.get("_cdc"); + assertThat(cdcMetadata.get("op")).isEqualTo("U"); + assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); + assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); + assertThat(cdcMetadata.get("key")).isInstanceOf(Map.class); + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); + } + } + + @Test + public void testDebeziumTransformWithSchemaAndKafkaMetadata() { + try (DebeziumTransform smt = new DebeziumTransform<>()) { + smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + Struct event = createDebeziumEventStruct("u"); Struct key = new Struct(KEY_SCHEMA).put("account_id", 1L); SinkRecord record = new SinkRecord("topic", 0, KEY_SCHEMA, key, VALUE_SCHEMA, event, 0); @@ -105,13 +152,9 @@ public void testDebeziumTransformWithSchema() { assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); - assertThat(value.get("account_id")).isEqualTo(1L); - - Struct cdcMetadata = value.getStruct("_cdc"); - assertThat(cdcMetadata.get("op")).isEqualTo("U"); - assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); - assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); - assertThat(cdcMetadata.get("key")).isInstanceOf(Struct.class); + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isInstanceOf(Struct.class); + Struct metadata = (Struct) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + assertThat(metadata.get("topic")).isEqualTo(record.topic()); } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java index 6bb2f20e..5641b010 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java @@ -22,6 +22,8 @@ import java.time.Instant; import java.util.Map; + +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -44,6 +46,25 @@ public void testDmsTransform() { Map cdcMetadata = (Map) value.get("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); assertThat(cdcMetadata.get("source")).isEqualTo("db.tbl"); + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); + } + } + + @Test + public void testDmsTransformWithkafkaMetadata() { + try (DmsTransform smt = new DmsTransform<>()) { + smt.configure(ImmutableMap.of( KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + + Map event = createDmsEvent("update"); + SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); + + SinkRecord result = smt.apply(record); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); + Map metadata = (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + assertThat(metadata.get("topic")).isEqualTo("topic"); } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java new file mode 100644 index 00000000..5378ce0f --- /dev/null +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java @@ -0,0 +1,142 @@ +package io.tabular.iceberg.connect.transforms.util; + +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.assertj.core.util.Maps; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.ExternalStringKafkaData; +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.EmptyExternalData; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; + + +public class KafkaMetadataAppenderTest { + + private static SchemaBuilder baseBuilder() { + return SchemaBuilder + .struct() + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + } + + private static final long TIMESTAMP = 100L; + private static final String TOPIC = "topic"; + private static final int PARTITION = 0; + private static final long OFFSET = 1000; + private static final Schema SCHEMA = baseBuilder().build(); + + private static final Schema SCHEMA_WITH_EXT_FIELD = baseBuilder().field("external", Schema.STRING_SCHEMA); + + private static final ExternalStringKafkaData EXT_FIELD = new ExternalStringKafkaData("external", "value"); + + private static final Schema RECORD_SCHEMA = SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA).build(); + + private static SinkRecord genSinkRecord(Boolean includeTimestamp) { + Struct value = new Struct(RECORD_SCHEMA); + value.put("id", "id"); + if (includeTimestamp) { + return new SinkRecord(TOPIC, PARTITION, null, null, RECORD_SCHEMA, value, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); + } else { + return new SinkRecord(TOPIC, PARTITION, null, null, RECORD_SCHEMA, value, OFFSET); + } + + } + + private static SinkRecord genSinkRecordAsMap(Boolean includeTimestamp) { + Map value = ImmutableMap.of("id", "id"); + if (includeTimestamp) { + return new SinkRecord(TOPIC, PARTITION, null, null, null, value, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); + } else { + return new SinkRecord(TOPIC, PARTITION, null, null, null, value, OFFSET); + } + } + + + @Test + @DisplayName("appendSchema should append the configured schema") + public void appendSchema() { + KafkaMetadataAppender appender = new KafkaMetadataAppender(new EmptyExternalData(), SCHEMA, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + Schema appendResult = appender.appendSchema(SchemaBuilder.struct()); + Schema appendExternalResult = appenderWithExternalField.appendSchema(SchemaBuilder.struct()); + + assertThat(appendResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()).isEqualTo(SCHEMA); + assertThat(appendExternalResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()).isEqualTo(SCHEMA_WITH_EXT_FIELD); + } + + + @Test + @DisplayName("appendToStruct should append record metadata under the configured key") + public void appendToStruct() { + KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + SinkRecord recordWithTimestamp = genSinkRecord(true); + SinkRecord recordWithoutTimestamp = genSinkRecord(false); + + Schema schema = appenderWithExternalField.appendSchema(SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA)); + + Struct resultParent = appenderWithExternalField.appendToStruct(recordWithTimestamp, new Struct(schema)); + Struct result = (Struct) resultParent.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + Struct resultWithoutTime = (Struct) appenderWithExternalField.appendToStruct(recordWithoutTimestamp, new Struct(schema)).get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + assertThat(resultParent.schema().field("id").schema().type()).isEqualTo(Schema.Type.STRING); + assertThat(result.get("external")).isEqualTo("value"); + assertThat(resultWithoutTime.get("external")).isEqualTo("value"); + assertThat(result.get("topic")).isEqualTo(TOPIC); + assertThat(resultWithoutTime.get("topic")).isEqualTo(TOPIC); + assertThat(result.get("partition")).isEqualTo(PARTITION); + assertThat(resultWithoutTime.get("partition")).isEqualTo(PARTITION); + assertThat(result.get("offset")).isEqualTo(OFFSET); + assertThat(resultWithoutTime.get("offset")).isEqualTo(OFFSET); + assertThat(result.get("timestamp")).isEqualTo(TIMESTAMP); + assertThat(resultWithoutTime.get("timestamp")).isNull(); + } + + @Test + @DisplayName("appendToMap should append record metadata under the configured key") + public void appendToMap() { + KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + SinkRecord recordWithTimestamp = genSinkRecordAsMap(true); + SinkRecord recordWithoutTimestamp = genSinkRecordAsMap(false); + + Map record1 = Maps.newHashMap("id", "id"); + Map record2 = Maps.newHashMap("id", "id"); + + Map expected = ImmutableMap.of( + "id", "id", + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, ImmutableMap.of( + "topic", TOPIC, + "partition", PARTITION, + "offset", OFFSET, + "timestamp", TIMESTAMP, + "external", "value" + ) + ); + + Map expectedWithoutTime = ImmutableMap.of( + "id", "id", + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, ImmutableMap.of( + "topic", TOPIC, + "partition", PARTITION, + "offset", OFFSET, + "external","value" + ) + ); + + Map result = appenderWithExternalField.appendToMap(recordWithTimestamp, record1); + Map resultWithoutTime = appenderWithExternalField.appendToMap(recordWithoutTimestamp,record2); + + assertThat(result).isEqualTo(expected); + assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); + } +} From 5788722485b448fd3a23ba97ca64b3dd4f66002d Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 16:24:18 -0800 Subject: [PATCH 02/17] formatting and docs --- kafka-connect-transforms/README.md | 19 +- .../connect/transforms/DebeziumTransform.java | 43 ++- .../connect/transforms/DmsTransform.java | 44 ++- .../util/KafkaMetadataAppender.java | 232 ++++++++------ .../transforms/DebeziumTransformTest.java | 23 +- .../connect/transforms/DmsTransformTest.java | 8 +- .../util/KafkaMetadataAppenderTest.java | 301 ++++++++++-------- 7 files changed, 381 insertions(+), 289 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index e8d22e45..e73433fd 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -33,7 +33,13 @@ It will promote the `data` element fields to top level and add the following met ## Configuration -The SMT currently has no configuration. +| Property | Description | +|:---------------------------|:---------------------------------------------------------------------------------------------| +| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | +| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | +| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | + +Kafka metadata injection only works on `SinkRecords` not `SourceRecords` # DebeziumTransform _(Experimental)_ @@ -44,6 +50,11 @@ It will promote the `before` or `after` element fields to top level and add the ## Configuration -| Property | Description | -|---------------------|-----------------------------------------------------------------------------------| -| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | +| Property | Description | +|:---------------------------|:---------------------------------------------------------------------------------------------| +| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | +| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | +| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | +| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | + +Kafka metadata injection only works on `SinkRecords` not `SourceRecords` diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 6ce9401f..744a0b61 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -18,10 +18,8 @@ */ package io.tabular.iceberg.connect.transforms; -import java.util.Map; -import java.util.Objects; - import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; +import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -57,22 +55,23 @@ public class DebeziumTransform> implements Transforma Importance.MEDIUM, "Pattern to use for setting the CDC target field value.") .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - Importance.LOW, - "Include appending of Kafka metadata to SinkRecord" - ) - .define(KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - Importance.LOW, - "field to append Kafka metadata under") - .define(KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); private String cdcTargetPattern; @@ -134,7 +133,7 @@ private R applyWithSchema(R record) { } newValue.put(CdcConstants.COL_CDC, cdcMetadata); - if (Objects.nonNull(kafkaAppender)) { + if (kafkaAppender != null) { if (record instanceof SinkRecord) { kafkaAppender.appendToStruct((SinkRecord) record, newValue); } @@ -177,7 +176,7 @@ private R applySchemaless(R record) { cdcMetadata.put(CdcConstants.COL_TS, value.get("ts_ms")); if (record instanceof SinkRecord) { cdcMetadata.put(CdcConstants.COL_OFFSET, ((SinkRecord) record).kafkaOffset()); - if (Objects.nonNull(kafkaAppender)) { + if (kafkaAppender != null) { kafkaAppender.appendToMap((SinkRecord) record, newValue); } } @@ -272,7 +271,7 @@ private Schema makeUpdatedSchema(Schema schema, Schema cdcSchema) { builder.field(CdcConstants.COL_CDC, cdcSchema); - if (Objects.nonNull(kafkaAppender)) { + if (kafkaAppender != null) { kafkaAppender.appendSchema(builder); } return builder.build(); diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index e50b8545..c98e6197 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -18,10 +18,8 @@ */ package io.tabular.iceberg.connect.transforms; -import java.util.Map; -import java.util.Objects; - import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; +import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; @@ -36,28 +34,28 @@ public class DmsTransform> implements Transformation< private static final Logger LOG = LoggerFactory.getLogger(DmsTransform.class.getName()); public static final ConfigDef CONFIG_DEF = - new ConfigDef() - .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - ConfigDef.Importance.LOW, - "Include appending of Kafka metadata to SinkRecord" - ) - .define(KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - ConfigDef.Importance.LOW, - "field to append Kafka metadata under") - .define(KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - ConfigDef.Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + new ConfigDef() + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); private static KafkaMetadataAppender kafkaAppender = null; - @Override public R apply(R record) { if (record.value() == null) { @@ -108,7 +106,7 @@ private R applySchemaless(R record) { Map newValue = Maps.newHashMap((Map) dataObj); newValue.put(CdcConstants.COL_CDC, cdcMetadata); - if(Objects.nonNull(kafkaAppender)) { + if (kafkaAppender != null) { if (record instanceof SinkRecord) { kafkaAppender.appendToMap((SinkRecord) record, newValue); } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java index f7cda1cd..13d588a3 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -1,5 +1,24 @@ +/* + * 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.util; +import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.data.Schema; @@ -8,129 +27,134 @@ import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; -import java.util.Map; -import java.util.Objects; - public class KafkaMetadataAppender { - public static final String INCLUDE_KAFKA_METADATA = "cdc.kafka.include_metadata"; - public static final String EXTERNAL_KAFKA_METADATA = "cdc.kafka.external_field"; - public static final String KEY_METADATA_FIELD_NAME = "cdc.kafka.metadata_field"; - public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; + public static final String INCLUDE_KAFKA_METADATA = "cdc.kafka.include_metadata"; + public static final String EXTERNAL_KAFKA_METADATA = "cdc.kafka.external_field"; + public static final String KEY_METADATA_FIELD_NAME = "cdc.kafka.metadata_field"; + public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; - private final ExternalKafkaData externalData; + private final ExternalKafkaData externalData; - private final Schema schema; + private final Schema schema; - private final String metadataFieldName; + private final String metadataFieldName; - public KafkaMetadataAppender(ExternalKafkaData externalData, Schema schema, String metadataFieldName) { - this.externalData = externalData; - this.schema = schema; - this.metadataFieldName = metadataFieldName; - } + public KafkaMetadataAppender( + ExternalKafkaData externalData, Schema schema, String metadataFieldName) { + this.externalData = externalData; + this.schema = schema; + this.metadataFieldName = metadataFieldName; + } + + public interface ExternalKafkaData { + SchemaBuilder addToSchema(SchemaBuilder builder); - public interface ExternalKafkaData { - SchemaBuilder addToSchema(SchemaBuilder builder); - Struct addToStruct(Struct struct); - Map addToMap(Map map); + Struct addToStruct(Struct struct); + + Map addToMap(Map map); + } + + public static class ExternalStringKafkaData implements ExternalKafkaData { + private final String name; + private final String value; + + public ExternalStringKafkaData(String name, String value) { + this.name = name; + this.value = value; } - public static class ExternalStringKafkaData implements ExternalKafkaData{ - private final String name; - private final String value; - - public ExternalStringKafkaData(String name, String value) { - this.name = name; - this.value = value; - } - - public static ExternalKafkaData parse(String field) { - if(field.equals("none")) { - return new EmptyExternalData(); - } - String[] parts = field.split(","); - if (parts.length != 2) { - throw new ConfigException(String.format("Could not parse %s for %s", field, KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA)); - } - return new ExternalStringKafkaData(parts[0], parts[1]); - } - - @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder.field(this.name, Schema.STRING_SCHEMA); - } - - @Override - public Struct addToStruct(Struct struct) { - return struct.put(this.name, this.value); - } - - @Override - public Map addToMap(Map map) { - map.put(this.name, this.value); - return map; - } + + public static ExternalKafkaData parse(String field) { + if (field.equals("none")) { + return new EmptyExternalData(); + } + String[] parts = field.split(","); + if (parts.length != 2) { + throw new ConfigException( + String.format( + "Could not parse %s for %s", field, KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA)); + } + return new ExternalStringKafkaData(parts[0], parts[1]); } - public static class EmptyExternalData implements ExternalKafkaData { - @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder; - } - - @Override - public Struct addToStruct(Struct struct) { - return struct; - } - - @Override - public Map addToMap(Map map) { - return map; - } + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(this.name, Schema.STRING_SCHEMA); } - public static KafkaMetadataAppender from(SimpleConfig config) { - ExternalKafkaData externalAppender = ExternalStringKafkaData.parse(config.getString(EXTERNAL_KAFKA_METADATA)); - String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME); + @Override + public Struct addToStruct(Struct struct) { + return struct.put(this.name, this.value); + } - SchemaBuilder schema = SchemaBuilder.struct(); + @Override + public Map addToMap(Map map) { + map.put(this.name, this.value); + return map; + } + } - externalAppender.addToSchema(schema) - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); - // TODO headers - return new KafkaMetadataAppender(externalAppender, schema.build(), metadataFieldName); + public static class EmptyExternalData implements ExternalKafkaData { + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder; } - public SchemaBuilder appendSchema(SchemaBuilder schema) { - return schema.field(this.metadataFieldName, this.schema); + @Override + public Struct addToStruct(Struct struct) { + return struct; } - public Struct appendToStruct(SinkRecord record, Struct struct) { - Struct metadata = new Struct(this.schema); - externalData.addToStruct(metadata); - metadata.put("topic", record.topic()); - metadata.put("partition", record.kafkaPartition()); - metadata.put("offset", record.kafkaOffset()); - if (Objects.nonNull(record.timestamp())) { - metadata.put("timestamp", record.timestamp()); - } - struct.put(this.metadataFieldName, metadata); - return struct; + @Override + public Map addToMap(Map map) { + return map; } + } + + public static KafkaMetadataAppender from(SimpleConfig config) { + ExternalKafkaData externalAppender = + ExternalStringKafkaData.parse(config.getString(EXTERNAL_KAFKA_METADATA)); + String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME); - public Map appendToMap(SinkRecord record, Map map) { - Map metadata = Maps.newHashMap(); - externalData.addToMap(metadata); - metadata.put("topic", record.topic()); - metadata.put("partition", record.kafkaPartition()); - metadata.put("offset", record.kafkaOffset()); - if (Objects.nonNull(record.timestamp())) { - metadata.put("timestamp", record.timestamp()); - } - map.put(this.metadataFieldName, metadata); - return map; + SchemaBuilder schema = SchemaBuilder.struct(); + + externalAppender + .addToSchema(schema) + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + // TODO headers + return new KafkaMetadataAppender(externalAppender, schema.build(), metadataFieldName); + } + + public SchemaBuilder appendSchema(SchemaBuilder builder) { + return builder.field(this.metadataFieldName, this.schema); + } + + public Struct appendToStruct(SinkRecord record, Struct struct) { + Struct metadata = new Struct(this.schema); + externalData.addToStruct(metadata); + metadata.put("topic", record.topic()); + metadata.put("partition", record.kafkaPartition()); + metadata.put("offset", record.kafkaOffset()); + if (record.timestamp() == null) { + metadata.put("timestamp", record.timestamp()); + } + struct.put(this.metadataFieldName, metadata); + return struct; + } + + public Map appendToMap(SinkRecord record, Map map) { + Map metadata = Maps.newHashMap(); + externalData.addToMap(metadata); + metadata.put("topic", record.topic()); + metadata.put("partition", record.kafkaPartition()); + metadata.put("offset", record.kafkaOffset()); + if (record.timestamp() == null) { + metadata.put("timestamp", record.timestamp()); } + map.put(this.metadataFieldName, metadata); + return map; + } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index e0410dbe..99b4645b 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -20,11 +20,10 @@ import static org.assertj.core.api.Assertions.assertThat; +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.math.BigDecimal; import java.time.Instant; import java.util.Map; - -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; @@ -99,7 +98,12 @@ public void testDebeziumTransformSchemaless() { @SuppressWarnings("unchecked") public void testDebeziumTransformSchemalessAndKafkaMetadata() { try (DebeziumTransform smt = new DebeziumTransform<>()) { - smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + smt.configure( + ImmutableMap.of( + "cdc.target.pattern", + "{db}_x.{table}_x", + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true)); Map event = createDebeziumEventMap("u"); Map key = ImmutableMap.of("account_id", 1L); @@ -110,7 +114,8 @@ public void testDebeziumTransformSchemalessAndKafkaMetadata() { Map value = (Map) result.value(); assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); - Map metadata = (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + Map metadata = + (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); assertThat(metadata.get("topic")).isEqualTo("topic"); } } @@ -142,7 +147,12 @@ public void testDebeziumTransformWithSchema() { @Test public void testDebeziumTransformWithSchemaAndKafkaMetadata() { try (DebeziumTransform smt = new DebeziumTransform<>()) { - smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + smt.configure( + ImmutableMap.of( + "cdc.target.pattern", + "{db}_x.{table}_x", + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true)); Struct event = createDebeziumEventStruct("u"); Struct key = new Struct(KEY_SCHEMA).put("account_id", 1L); @@ -152,7 +162,8 @@ public void testDebeziumTransformWithSchemaAndKafkaMetadata() { assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isInstanceOf(Struct.class); + assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)) + .isInstanceOf(Struct.class); Struct metadata = (Struct) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); assertThat(metadata.get("topic")).isEqualTo(record.topic()); } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java index 5641b010..d121be4f 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java @@ -20,10 +20,9 @@ import static org.assertj.core.api.Assertions.assertThat; +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.time.Instant; import java.util.Map; - -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -53,7 +52,7 @@ public void testDmsTransform() { @Test public void testDmsTransformWithkafkaMetadata() { try (DmsTransform smt = new DmsTransform<>()) { - smt.configure(ImmutableMap.of( KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + smt.configure(ImmutableMap.of(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); Map event = createDmsEvent("update"); SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); @@ -63,7 +62,8 @@ public void testDmsTransformWithkafkaMetadata() { Map value = (Map) result.value(); assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); - Map metadata = (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + Map metadata = + (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); assertThat(metadata.get("topic")).isEqualTo("topic"); } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java index 5378ce0f..3d41bb52 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java @@ -1,142 +1,191 @@ +/* + * 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.util; +import static org.assertj.core.api.Assertions.assertThat; + +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.EmptyExternalData; +import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.ExternalStringKafkaData; +import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; 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.assertj.core.util.Maps; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.ExternalStringKafkaData; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.EmptyExternalData; -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; - public class KafkaMetadataAppenderTest { - private static SchemaBuilder baseBuilder() { - return SchemaBuilder - .struct() - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + private static SchemaBuilder baseBuilder() { + return SchemaBuilder.struct() + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + } + + private static final long TIMESTAMP = 100L; + private static final String TOPIC = "topic"; + private static final int PARTITION = 0; + private static final long OFFSET = 1000; + private static final Schema SCHEMA = baseBuilder().build(); + + private static final Schema SCHEMA_WITH_EXT_FIELD = + baseBuilder().field("external", Schema.STRING_SCHEMA); + + private static final ExternalStringKafkaData EXT_FIELD = + new ExternalStringKafkaData("external", "value"); + + private static final Schema RECORD_SCHEMA = + SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA).build(); + + private static SinkRecord genSinkRecord(Boolean includeTimestamp) { + Struct value = new Struct(RECORD_SCHEMA); + value.put("id", "id"); + if (includeTimestamp) { + return new SinkRecord( + TOPIC, + PARTITION, + null, + null, + RECORD_SCHEMA, + value, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME); + } else { + return new SinkRecord(TOPIC, PARTITION, null, null, RECORD_SCHEMA, value, OFFSET); } - - private static final long TIMESTAMP = 100L; - private static final String TOPIC = "topic"; - private static final int PARTITION = 0; - private static final long OFFSET = 1000; - private static final Schema SCHEMA = baseBuilder().build(); - - private static final Schema SCHEMA_WITH_EXT_FIELD = baseBuilder().field("external", Schema.STRING_SCHEMA); - - private static final ExternalStringKafkaData EXT_FIELD = new ExternalStringKafkaData("external", "value"); - - private static final Schema RECORD_SCHEMA = SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA).build(); - - private static SinkRecord genSinkRecord(Boolean includeTimestamp) { - Struct value = new Struct(RECORD_SCHEMA); - value.put("id", "id"); - if (includeTimestamp) { - return new SinkRecord(TOPIC, PARTITION, null, null, RECORD_SCHEMA, value, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - } else { - return new SinkRecord(TOPIC, PARTITION, null, null, RECORD_SCHEMA, value, OFFSET); - } - - } - - private static SinkRecord genSinkRecordAsMap(Boolean includeTimestamp) { - Map value = ImmutableMap.of("id", "id"); - if (includeTimestamp) { - return new SinkRecord(TOPIC, PARTITION, null, null, null, value, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - } else { - return new SinkRecord(TOPIC, PARTITION, null, null, null, value, OFFSET); - } - } - - - @Test - @DisplayName("appendSchema should append the configured schema") - public void appendSchema() { - KafkaMetadataAppender appender = new KafkaMetadataAppender(new EmptyExternalData(), SCHEMA, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - - Schema appendResult = appender.appendSchema(SchemaBuilder.struct()); - Schema appendExternalResult = appenderWithExternalField.appendSchema(SchemaBuilder.struct()); - - assertThat(appendResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()).isEqualTo(SCHEMA); - assertThat(appendExternalResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()).isEqualTo(SCHEMA_WITH_EXT_FIELD); - } - - - @Test - @DisplayName("appendToStruct should append record metadata under the configured key") - public void appendToStruct() { - KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - - SinkRecord recordWithTimestamp = genSinkRecord(true); - SinkRecord recordWithoutTimestamp = genSinkRecord(false); - - Schema schema = appenderWithExternalField.appendSchema(SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA)); - - Struct resultParent = appenderWithExternalField.appendToStruct(recordWithTimestamp, new Struct(schema)); - Struct result = (Struct) resultParent.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - Struct resultWithoutTime = (Struct) appenderWithExternalField.appendToStruct(recordWithoutTimestamp, new Struct(schema)).get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - - assertThat(resultParent.schema().field("id").schema().type()).isEqualTo(Schema.Type.STRING); - assertThat(result.get("external")).isEqualTo("value"); - assertThat(resultWithoutTime.get("external")).isEqualTo("value"); - assertThat(result.get("topic")).isEqualTo(TOPIC); - assertThat(resultWithoutTime.get("topic")).isEqualTo(TOPIC); - assertThat(result.get("partition")).isEqualTo(PARTITION); - assertThat(resultWithoutTime.get("partition")).isEqualTo(PARTITION); - assertThat(result.get("offset")).isEqualTo(OFFSET); - assertThat(resultWithoutTime.get("offset")).isEqualTo(OFFSET); - assertThat(result.get("timestamp")).isEqualTo(TIMESTAMP); - assertThat(resultWithoutTime.get("timestamp")).isNull(); - } - - @Test - @DisplayName("appendToMap should append record metadata under the configured key") - public void appendToMap() { - KafkaMetadataAppender appenderWithExternalField = new KafkaMetadataAppender(EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - - SinkRecord recordWithTimestamp = genSinkRecordAsMap(true); - SinkRecord recordWithoutTimestamp = genSinkRecordAsMap(false); - - Map record1 = Maps.newHashMap("id", "id"); - Map record2 = Maps.newHashMap("id", "id"); - - Map expected = ImmutableMap.of( - "id", "id", - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, ImmutableMap.of( - "topic", TOPIC, - "partition", PARTITION, - "offset", OFFSET, - "timestamp", TIMESTAMP, - "external", "value" - ) - ); - - Map expectedWithoutTime = ImmutableMap.of( - "id", "id", - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, ImmutableMap.of( - "topic", TOPIC, - "partition", PARTITION, - "offset", OFFSET, - "external","value" - ) - ); - - Map result = appenderWithExternalField.appendToMap(recordWithTimestamp, record1); - Map resultWithoutTime = appenderWithExternalField.appendToMap(recordWithoutTimestamp,record2); - - assertThat(result).isEqualTo(expected); - assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); + } + + private static SinkRecord genSinkRecordAsMap(Boolean includeTimestamp) { + Map value = ImmutableMap.of("id", "id"); + if (includeTimestamp) { + return new SinkRecord( + TOPIC, PARTITION, null, null, null, value, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); + } else { + return new SinkRecord(TOPIC, PARTITION, null, null, null, value, OFFSET); } + } + + @Test + @DisplayName("appendSchema should append the configured schema") + public void appendSchema() { + KafkaMetadataAppender appender = + new KafkaMetadataAppender( + new EmptyExternalData(), SCHEMA, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + KafkaMetadataAppender appenderWithExternalField = + new KafkaMetadataAppender( + EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + Schema appendResult = appender.appendSchema(SchemaBuilder.struct()); + Schema appendExternalResult = appenderWithExternalField.appendSchema(SchemaBuilder.struct()); + + assertThat(appendResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()) + .isEqualTo(SCHEMA); + assertThat( + appendExternalResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()) + .isEqualTo(SCHEMA_WITH_EXT_FIELD); + } + + @Test + @DisplayName("appendToStruct should append record metadata under the configured key") + public void appendToStruct() { + KafkaMetadataAppender appenderWithExternalField = + new KafkaMetadataAppender( + EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + SinkRecord recordWithTimestamp = genSinkRecord(true); + SinkRecord recordWithoutTimestamp = genSinkRecord(false); + + Schema schema = + appenderWithExternalField.appendSchema( + SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA)); + + Struct resultParent = + appenderWithExternalField.appendToStruct(recordWithTimestamp, new Struct(schema)); + Struct result = (Struct) resultParent.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + Struct resultWithoutTime = + (Struct) + appenderWithExternalField + .appendToStruct(recordWithoutTimestamp, new Struct(schema)) + .get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + assertThat(resultParent.schema().field("id").schema().type()).isEqualTo(Schema.Type.STRING); + assertThat(result.get("external")).isEqualTo("value"); + assertThat(resultWithoutTime.get("external")).isEqualTo("value"); + assertThat(result.get("topic")).isEqualTo(TOPIC); + assertThat(resultWithoutTime.get("topic")).isEqualTo(TOPIC); + assertThat(result.get("partition")).isEqualTo(PARTITION); + assertThat(resultWithoutTime.get("partition")).isEqualTo(PARTITION); + assertThat(result.get("offset")).isEqualTo(OFFSET); + assertThat(resultWithoutTime.get("offset")).isEqualTo(OFFSET); + assertThat(result.get("timestamp")).isEqualTo(TIMESTAMP); + assertThat(resultWithoutTime.get("timestamp")).isNull(); + } + + @Test + @DisplayName("appendToMap should append record metadata under the configured key") + public void appendToMap() { + KafkaMetadataAppender appenderWithExternalField = + new KafkaMetadataAppender( + EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + + SinkRecord recordWithTimestamp = genSinkRecordAsMap(true); + SinkRecord recordWithoutTimestamp = genSinkRecordAsMap(false); + + Map record1 = Maps.newHashMap(); + record1.put("id", "id"); + Map record2 = Maps.newHashMap(); + record2.put("id", "id"); + Map expected = + ImmutableMap.of( + "id", + "id", + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ImmutableMap.of( + "topic", TOPIC, + "partition", PARTITION, + "offset", OFFSET, + "timestamp", TIMESTAMP, + "external", "value")); + + Map expectedWithoutTime = + ImmutableMap.of( + "id", + "id", + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ImmutableMap.of( + "topic", TOPIC, + "partition", PARTITION, + "offset", OFFSET, + "external", "value")); + + Map result = + appenderWithExternalField.appendToMap(recordWithTimestamp, record1); + Map resultWithoutTime = + appenderWithExternalField.appendToMap(recordWithoutTimestamp, record2); + + assertThat(result).isEqualTo(expected); + assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); + } } From 0cb9d85fc5ed6dd7f878e05b4e4d12dd32c672c8 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 16:40:47 -0800 Subject: [PATCH 03/17] fix null checks --- .../connect/transforms/util/KafkaMetadataAppender.java | 4 ++-- .../iceberg/connect/transforms/DebeziumTransformTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java index 13d588a3..8a31cca6 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -138,7 +138,7 @@ public Struct appendToStruct(SinkRecord record, Struct struct) { metadata.put("topic", record.topic()); metadata.put("partition", record.kafkaPartition()); metadata.put("offset", record.kafkaOffset()); - if (record.timestamp() == null) { + if (record.timestamp() != null) { metadata.put("timestamp", record.timestamp()); } struct.put(this.metadataFieldName, metadata); @@ -151,7 +151,7 @@ public Map appendToMap(SinkRecord record, Map ma metadata.put("topic", record.topic()); metadata.put("partition", record.kafkaPartition()); metadata.put("offset", record.kafkaOffset()); - if (record.timestamp() == null) { + if (record.timestamp() != null) { metadata.put("timestamp", record.timestamp()); } map.put(this.metadataFieldName, metadata); diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index 99b4645b..a0a9b100 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -140,7 +140,7 @@ public void testDebeziumTransformWithSchema() { assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); assertThat(cdcMetadata.get("key")).isInstanceOf(Map.class); - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); + assertThat(value.keySet()).doesNotContain(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); } } From 800de10a8eb164a55561dbd75ed03629937ce382 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 16:58:08 -0800 Subject: [PATCH 04/17] tests --- .../transforms/DebeziumTransformTest.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index a0a9b100..e550334a 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -19,6 +19,7 @@ package io.tabular.iceberg.connect.transforms; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.math.BigDecimal; @@ -29,6 +30,7 @@ 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.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -125,22 +127,23 @@ public void testDebeziumTransformWithSchema() { try (DebeziumTransform smt = new DebeziumTransform<>()) { smt.configure(ImmutableMap.of("cdc.target.pattern", "{db}_x.{table}_x")); - Map event = createDebeziumEventMap("u"); - Map key = ImmutableMap.of("account_id", 1L); - SinkRecord record = new SinkRecord("topic", 0, null, key, null, event, 0); + Struct event = createDebeziumEventStruct("u"); + Struct key = new Struct(KEY_SCHEMA).put("account_id", 1L); + SinkRecord record = new SinkRecord("topic", 0, KEY_SCHEMA, key, VALUE_SCHEMA, event, 0); SinkRecord result = smt.apply(record); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); - assertThat(value.get("account_id")).isEqualTo(1); + assertThat(value.get("account_id")).isEqualTo(1L); + assertThrows( + DataException.class, () -> value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)); - Map cdcMetadata = (Map) value.get("_cdc"); + Struct cdcMetadata = value.getStruct("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); - assertThat(cdcMetadata.get("key")).isInstanceOf(Map.class); - assertThat(value.keySet()).doesNotContain(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + assertThat(cdcMetadata.get("key")).isInstanceOf(Struct.class); } } From 7600e099c206c5bbaed1dd576592ab776668eec0 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 17:15:57 -0800 Subject: [PATCH 05/17] flakey test --- .../iceberg/connect/transforms/DebeziumTransformTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index e550334a..29782e8a 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -19,7 +19,6 @@ package io.tabular.iceberg.connect.transforms; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.math.BigDecimal; @@ -30,7 +29,6 @@ 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.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -136,8 +134,6 @@ public void testDebeziumTransformWithSchema() { Struct value = (Struct) result.value(); assertThat(value.get("account_id")).isEqualTo(1L); - assertThrows( - DataException.class, () -> value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)); Struct cdcMetadata = value.getStruct("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); From 86486a9af7a44b5c0908e4ef0548d10e61f942d2 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 8 Mar 2024 17:20:05 -0800 Subject: [PATCH 06/17] fix final static bug causing test failures --- .../tabular/iceberg/connect/transforms/DebeziumTransform.java | 2 +- .../io/tabular/iceberg/connect/transforms/DmsTransform.java | 2 +- .../iceberg/connect/transforms/DebeziumTransformTest.java | 4 ++++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 744a0b61..419c4834 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -44,7 +44,7 @@ public class DebeziumTransform> implements Transforma private static final String CDC_TARGET_PATTERN = "cdc.target.pattern"; private static final String DB_PLACEHOLDER = "{db}"; private static final String TABLE_PLACEHOLDER = "{table}"; - private static KafkaMetadataAppender kafkaAppender = null; + private KafkaMetadataAppender kafkaAppender = null; public static final ConfigDef CONFIG_DEF = new ConfigDef() diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index c98e6197..877987e8 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -54,7 +54,7 @@ public class DmsTransform> implements Transformation< ConfigDef.Importance.LOW, "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); - private static KafkaMetadataAppender kafkaAppender = null; + private KafkaMetadataAppender kafkaAppender = null; @Override public R apply(R record) { diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index 29782e8a..c9256021 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -19,6 +19,7 @@ package io.tabular.iceberg.connect.transforms; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.math.BigDecimal; @@ -29,6 +30,7 @@ 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.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -134,6 +136,8 @@ public void testDebeziumTransformWithSchema() { Struct value = (Struct) result.value(); assertThat(value.get("account_id")).isEqualTo(1L); + assertThrows( + DataException.class, () -> value.get(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)); Struct cdcMetadata = value.getStruct("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); From 2fc31b5da04336aa422ed64573bccd348f1532f7 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 10 Mar 2024 21:17:38 -0700 Subject: [PATCH 07/17] reworked to allow nesting or flattening of kafka metadata --- kafka-connect-transforms/README.md | 32 +- .../connect/transforms/DebeziumTransform.java | 41 +-- .../connect/transforms/DmsTransform.java | 38 +-- .../util/KafkaMetadataAppender.java | 287 ++++++++++++------ .../transforms/util/RecordAppender.java | 33 ++ .../transforms/DebeziumTransformTest.java | 4 + .../connect/transforms/DmsTransformTest.java | 8 +- .../util/KafkaMetadataAppenderTest.java | 67 ++-- 8 files changed, 296 insertions(+), 214 deletions(-) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/RecordAppender.java diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index e73433fd..0ac68d71 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -33,13 +33,12 @@ It will promote the `data` element fields to top level and add the following met ## Configuration -| Property | Description | -|:---------------------------|:---------------------------------------------------------------------------------------------| -| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | -| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | -| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | - -Kafka metadata injection only works on `SinkRecords` not `SourceRecords` +| Property | Description | +|:-----------------------------|:--------------------------------------------------------------------------------------------------------| +| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | +| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | +| cdc.kafka.metadata_is_nested | If true nests metadata on a struct under field name, otherwise prefixes metadata fields with field name | +| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | # DebeziumTransform _(Experimental)_ @@ -48,13 +47,18 @@ The `DebeziumTransform` SMT transforms a Debezium formatted message for use by t It will promote the `before` or `after` element fields to top level and add the following metadata fields: `_cdc.op`, `_cdc.ts`, `_cdc.offset`, `_cdc.source`, `_cdc.target`, and `_cdc.key`. +If `cdc.kafka.include_metadata` it will add the following metadata fields: +`_kafka_metadata_topic`, `kafka_metadata_partition`, `kafka_metadata_offset`, and `kafka_metadata_record_timestamp`. + ## Configuration -| Property | Description | -|:---------------------------|:---------------------------------------------------------------------------------------------| -| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | -| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | -| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | -| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | +| Property | Description | +|:-----------------------------|:--------------------------------------------------------------------------------------------------------| +| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | +| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | +| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | +| cdc.kafka.metadata_is_nested | If true nests metadata on a struct under field name, otherwise prefixes metadata fields with field name | +| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | -Kafka metadata injection only works on `SinkRecords` not `SourceRecords` +If `cdc.kafka.include_metadata` it will add the following metadata fields: +`_kafka_metadata_topic`, `kafka_metadata_partition`, `kafka_metadata_offset`, and `kafka_metadata_record_timestamp`. \ No newline at end of file diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 419c4834..4d80cc50 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -19,6 +19,7 @@ package io.tabular.iceberg.connect.transforms; import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; +import io.tabular.iceberg.connect.transforms.util.RecordAppender; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; @@ -44,7 +45,7 @@ public class DebeziumTransform> implements Transforma private static final String CDC_TARGET_PATTERN = "cdc.target.pattern"; private static final String DB_PLACEHOLDER = "{db}"; private static final String TABLE_PLACEHOLDER = "{table}"; - private KafkaMetadataAppender kafkaAppender = null; + private RecordAppender kafkaAppender; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -53,25 +54,7 @@ public class DebeziumTransform> implements Transforma ConfigDef.Type.STRING, null, Importance.MEDIUM, - "Pattern to use for setting the CDC target field value.") - .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - Importance.LOW, - "Include appending of Kafka metadata to SinkRecord") - .define( - KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - Importance.LOW, - "field to append Kafka metadata under") - .define( - KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + "Pattern to use for setting the CDC target field value."); private String cdcTargetPattern; @@ -79,9 +62,7 @@ public class DebeziumTransform> implements Transforma public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); cdcTargetPattern = config.getString(CDC_TARGET_PATTERN); - if (config.getBoolean(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)) { - kafkaAppender = KafkaMetadataAppender.from(config); - } + kafkaAppender = KafkaMetadataAppender.from(props); } @Override @@ -133,11 +114,7 @@ private R applyWithSchema(R record) { } newValue.put(CdcConstants.COL_CDC, cdcMetadata); - if (kafkaAppender != null) { - if (record instanceof SinkRecord) { - kafkaAppender.appendToStruct((SinkRecord) record, newValue); - } - } + kafkaAppender.addToStruct(record, newValue); return record.newRecord( record.topic(), @@ -176,10 +153,8 @@ private R applySchemaless(R record) { cdcMetadata.put(CdcConstants.COL_TS, value.get("ts_ms")); if (record instanceof SinkRecord) { cdcMetadata.put(CdcConstants.COL_OFFSET, ((SinkRecord) record).kafkaOffset()); - if (kafkaAppender != null) { - kafkaAppender.appendToMap((SinkRecord) record, newValue); - } } + kafkaAppender.addToMap(record, newValue); setTableAndTargetFromSourceMap(value.get("source"), cdcMetadata); if (record.key() instanceof Map) { @@ -270,10 +245,8 @@ private Schema makeUpdatedSchema(Schema schema, Schema cdcSchema) { } builder.field(CdcConstants.COL_CDC, cdcSchema); + kafkaAppender.addToSchema(builder); - if (kafkaAppender != null) { - kafkaAppender.appendSchema(builder); - } return builder.build(); } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index 877987e8..fe4132d5 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -19,42 +19,21 @@ package io.tabular.iceberg.connect.transforms; import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; +import io.tabular.iceberg.connect.transforms.util.RecordAppender; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; -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.SimpleConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DmsTransform> implements Transformation { private static final Logger LOG = LoggerFactory.getLogger(DmsTransform.class.getName()); - public static final ConfigDef CONFIG_DEF = - new ConfigDef() - .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - ConfigDef.Importance.LOW, - "Include appending of Kafka metadata to SinkRecord") - .define( - KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - ConfigDef.Importance.LOW, - "field to append Kafka metadata under") - .define( - KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - ConfigDef.Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); - - private KafkaMetadataAppender kafkaAppender = null; + public static final ConfigDef CONFIG_DEF = new ConfigDef(); + private RecordAppender kafkaAppender = null; @Override public R apply(R record) { @@ -106,11 +85,7 @@ private R applySchemaless(R record) { Map newValue = Maps.newHashMap((Map) dataObj); newValue.put(CdcConstants.COL_CDC, cdcMetadata); - if (kafkaAppender != null) { - if (record instanceof SinkRecord) { - kafkaAppender.appendToMap((SinkRecord) record, newValue); - } - } + this.kafkaAppender.addToMap(record, newValue); return record.newRecord( record.topic(), @@ -132,9 +107,6 @@ public void close() {} @Override public void configure(Map configs) { - SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); - if (config.getBoolean(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)) { - kafkaAppender = KafkaMetadataAppender.from(config); - } + this.kafkaAppender = KafkaMetadataAppender.from(configs); } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java index 8a31cca6..5b8ad0d8 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -19,142 +19,227 @@ package io.tabular.iceberg.connect.transforms.util; 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.connector.ConnectRecord; 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.util.SimpleConfig; -public class KafkaMetadataAppender { +public abstract class KafkaMetadataAppender { public static final String INCLUDE_KAFKA_METADATA = "cdc.kafka.include_metadata"; public static final String EXTERNAL_KAFKA_METADATA = "cdc.kafka.external_field"; public static final String KEY_METADATA_FIELD_NAME = "cdc.kafka.metadata_field"; + public static final String KEY_METADATA_IS_NESTED = "cdc.kafka.metadata_is_nested"; public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; - private final ExternalKafkaData externalData; - - private final Schema schema; - - private final String metadataFieldName; - - public KafkaMetadataAppender( - ExternalKafkaData externalData, Schema schema, String metadataFieldName) { - this.externalData = externalData; - this.schema = schema; - this.metadataFieldName = metadataFieldName; - } - - public interface ExternalKafkaData { - SchemaBuilder addToSchema(SchemaBuilder builder); - - Struct addToStruct(Struct struct); - - Map addToMap(Map map); - } - - public static class ExternalStringKafkaData implements ExternalKafkaData { - private final String name; - private final String value; - - public ExternalStringKafkaData(String name, String value) { - this.name = name; - this.value = value; - } - - public static ExternalKafkaData parse(String field) { - if (field.equals("none")) { - return new EmptyExternalData(); - } - String[] parts = field.split(","); - if (parts.length != 2) { - throw new ConfigException( - String.format( - "Could not parse %s for %s", field, KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA)); - } - return new ExternalStringKafkaData(parts[0], parts[1]); - } - - @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder.field(this.name, Schema.STRING_SCHEMA); - } - - @Override - public Struct addToStruct(Struct struct) { - return struct.put(this.name, this.value); + public static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.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( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + + public static > RecordAppender from(Map props) { + SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + if (config.getBoolean(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)) { + return from(config); + } else { + return new NoOpRecordAppender(); } + } - @Override - public Map addToMap(Map map) { - map.put(this.name, this.value); - return map; + private static > RecordAppender from(SimpleConfig config) { + RecordAppender externalFieldAppender = + getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA)); + 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) { + topicFieldName = "topic"; + partitionFieldName = "partition"; + offsetFieldName = "offset"; + timestampFieldName = "record_timestamp"; + + SchemaBuilder nestedSchemaBuilder = SchemaBuilder.struct(); + nestedSchemaBuilder + .field(topicFieldName, Schema.STRING_SCHEMA) + .field(partitionFieldName, Schema.INT32_SCHEMA) + .field(offsetFieldName, Schema.OPTIONAL_INT64_SCHEMA) + .field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA); + externalFieldAppender.addToSchema(nestedSchemaBuilder); + + Schema nestedSchema = nestedSchemaBuilder.build(); + + return new RecordAppender() { + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(metadataFieldName, nestedSchema); + } + + @Override + public Struct addToStruct(R record, Struct struct) { + Struct nested = new Struct(nestedSchema); + nested.put(topicFieldName, record.topic()); + nested.put(partitionFieldName, record.kafkaPartition()); + if (record instanceof SinkRecord) { + SinkRecord sinkRecord = (SinkRecord) record; + nested.put(offsetFieldName, sinkRecord.kafkaOffset()); + } + if (record.timestamp() != null) { + nested.put(timestampFieldName, record.timestamp()); + } + externalFieldAppender.addToStruct(record, nested); + struct.put(metadataFieldName, nested); + return struct; + } + + @Override + public Map addToMap(R record, Map map) { + Map nested = Maps.newHashMap(); + nested.put("topic", record.topic()); + nested.put("partition", record.kafkaPartition()); + if (record instanceof SinkRecord) { + SinkRecord sinkRecord = (SinkRecord) record; + nested.put("offset", sinkRecord.kafkaOffset()); + } + if (record.timestamp() != null) { + nested.put("record_timestamp", record.timestamp()); + } + externalFieldAppender.addToMap(record, nested); + map.put(metadataFieldName, nested); + return map; + } + }; + + } 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("record_timestamp"); + + return new RecordAppender() { + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + builder + .field(topicFieldName, Schema.STRING_SCHEMA) + .field(partitionFieldName, Schema.INT32_SCHEMA) + .field(offsetFieldName, Schema.OPTIONAL_INT64_SCHEMA) + .field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA); + return externalFieldAppender.addToSchema(builder); + } + + @Override + public Struct addToStruct(R record, Struct struct) { + struct.put(topicFieldName, record.topic()); + struct.put(partitionFieldName, record.kafkaPartition()); + if (record instanceof SinkRecord) { + SinkRecord sinkRecord = (SinkRecord) record; + struct.put(offsetFieldName, sinkRecord.kafkaOffset()); + } + if (record.timestamp() != null) { + struct.put(timestampFieldName, record.timestamp()); + } + externalFieldAppender.addToStruct(record, struct); + return struct; + } + + @Override + public Map addToMap(R record, Map map) { + map.put("topic", record.topic()); + map.put("partition", record.kafkaPartition()); + if (record instanceof SinkRecord) { + SinkRecord sinkRecord = (SinkRecord) record; + map.put("offset", sinkRecord.kafkaOffset()); + } + if (record.timestamp() != null) { + map.put("record_timestamp", record.timestamp()); + } + externalFieldAppender.addToMap(record, map); + return map; + } + }; } } - public static class EmptyExternalData implements ExternalKafkaData { + public static class NoOpRecordAppender> implements RecordAppender { + @Override public SchemaBuilder addToSchema(SchemaBuilder builder) { return builder; } @Override - public Struct addToStruct(Struct struct) { + public Struct addToStruct(R record, Struct struct) { return struct; } @Override - public Map addToMap(Map map) { + public Map addToMap(R record, Map map) { return map; } } - public static KafkaMetadataAppender from(SimpleConfig config) { - ExternalKafkaData externalAppender = - ExternalStringKafkaData.parse(config.getString(EXTERNAL_KAFKA_METADATA)); - String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME); - - SchemaBuilder schema = SchemaBuilder.struct(); - - externalAppender - .addToSchema(schema) - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); - // TODO headers - return new KafkaMetadataAppender(externalAppender, schema.build(), metadataFieldName); - } + private static > RecordAppender getExternalFieldAppender( + String field) { + if (field.equals("none")) { + 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 = parts[0]; + String fieldValue = parts[1]; + return new RecordAppender() { - public SchemaBuilder appendSchema(SchemaBuilder builder) { - return builder.field(this.metadataFieldName, this.schema); - } + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(fieldName, Schema.STRING_SCHEMA); + } - public Struct appendToStruct(SinkRecord record, Struct struct) { - Struct metadata = new Struct(this.schema); - externalData.addToStruct(metadata); - metadata.put("topic", record.topic()); - metadata.put("partition", record.kafkaPartition()); - metadata.put("offset", record.kafkaOffset()); - if (record.timestamp() != null) { - metadata.put("timestamp", record.timestamp()); - } - struct.put(this.metadataFieldName, metadata); - return struct; - } + @Override + public Struct addToStruct(R record, Struct struct) { + return struct.put(fieldName, fieldValue); + } - public Map appendToMap(SinkRecord record, Map map) { - Map metadata = Maps.newHashMap(); - externalData.addToMap(metadata); - metadata.put("topic", record.topic()); - metadata.put("partition", record.kafkaPartition()); - metadata.put("offset", record.kafkaOffset()); - if (record.timestamp() != null) { - metadata.put("timestamp", record.timestamp()); - } - map.put(this.metadataFieldName, metadata); - return map; + @Override + public Map addToMap(R record, Map map) { + map.put(fieldName, fieldValue); + return map; + } + }; } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/RecordAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/RecordAppender.java new file mode 100644 index 00000000..2f30052a --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/RecordAppender.java @@ -0,0 +1,33 @@ +/* + * 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.util; + +import java.util.Map; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; + +public interface RecordAppender> { + + SchemaBuilder addToSchema(SchemaBuilder builder); + + Struct addToStruct(R record, Struct struct); + + Map addToMap(R record, Map map); +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index c9256021..f652819c 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -105,6 +105,8 @@ public void testDebeziumTransformSchemalessAndKafkaMetadata() { "cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true)); Map event = createDebeziumEventMap("u"); @@ -155,6 +157,8 @@ public void testDebeziumTransformWithSchemaAndKafkaMetadata() { "cdc.target.pattern", "{db}_x.{table}_x", KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true)); Struct event = createDebeziumEventStruct("u"); diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java index d121be4f..79900da8 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java @@ -33,6 +33,7 @@ public class DmsTransformTest { @SuppressWarnings("unchecked") public void testDmsTransform() { try (DmsTransform smt = new DmsTransform<>()) { + smt.configure(ImmutableMap.of()); Map event = createDmsEvent("update"); SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); @@ -52,7 +53,12 @@ public void testDmsTransform() { @Test public void testDmsTransformWithkafkaMetadata() { try (DmsTransform smt = new DmsTransform<>()) { - smt.configure(ImmutableMap.of(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true)); + smt.configure( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, + true)); Map event = createDmsEvent("update"); SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java index 3d41bb52..8ce8641a 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java @@ -20,8 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.EmptyExternalData; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender.ExternalStringKafkaData; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -39,8 +37,8 @@ private static SchemaBuilder baseBuilder() { return SchemaBuilder.struct() .field("topic", Schema.STRING_SCHEMA) .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA); + .field("offset", Schema.OPTIONAL_INT64_SCHEMA) + .field("record_timestamp", Schema.OPTIONAL_INT64_SCHEMA); } private static final long TIMESTAMP = 100L; @@ -50,10 +48,7 @@ private static SchemaBuilder baseBuilder() { private static final Schema SCHEMA = baseBuilder().build(); private static final Schema SCHEMA_WITH_EXT_FIELD = - baseBuilder().field("external", Schema.STRING_SCHEMA); - - private static final ExternalStringKafkaData EXT_FIELD = - new ExternalStringKafkaData("external", "value"); + baseBuilder().field("external", Schema.STRING_SCHEMA).build(); private static final Schema RECORD_SCHEMA = SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA).build(); @@ -90,15 +85,20 @@ private static SinkRecord genSinkRecordAsMap(Boolean includeTimestamp) { @Test @DisplayName("appendSchema should append the configured schema") public void appendSchema() { - KafkaMetadataAppender appender = - new KafkaMetadataAppender( - new EmptyExternalData(), SCHEMA, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - KafkaMetadataAppender appenderWithExternalField = - new KafkaMetadataAppender( - EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + RecordAppender appender = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true)); + RecordAppender appenderWithExternalField = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true, + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, "external,value")); - Schema appendResult = appender.appendSchema(SchemaBuilder.struct()); - Schema appendExternalResult = appenderWithExternalField.appendSchema(SchemaBuilder.struct()); + Schema appendResult = appender.addToSchema(SchemaBuilder.struct()); + Schema appendExternalResult = appenderWithExternalField.addToSchema(SchemaBuilder.struct()); assertThat(appendResult.field(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME).schema()) .isEqualTo(SCHEMA); @@ -110,24 +110,27 @@ public void appendSchema() { @Test @DisplayName("appendToStruct should append record metadata under the configured key") public void appendToStruct() { - KafkaMetadataAppender appenderWithExternalField = - new KafkaMetadataAppender( - EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + RecordAppender appenderWithExternalField = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true, + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, "external,value")); SinkRecord recordWithTimestamp = genSinkRecord(true); SinkRecord recordWithoutTimestamp = genSinkRecord(false); Schema schema = - appenderWithExternalField.appendSchema( + appenderWithExternalField.addToSchema( SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA)); Struct resultParent = - appenderWithExternalField.appendToStruct(recordWithTimestamp, new Struct(schema)); + appenderWithExternalField.addToStruct(recordWithTimestamp, new Struct(schema)); Struct result = (Struct) resultParent.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); Struct resultWithoutTime = (Struct) appenderWithExternalField - .appendToStruct(recordWithoutTimestamp, new Struct(schema)) + .addToStruct(recordWithoutTimestamp, new Struct(schema)) .get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); assertThat(resultParent.schema().field("id").schema().type()).isEqualTo(Schema.Type.STRING); @@ -139,16 +142,19 @@ public void appendToStruct() { assertThat(resultWithoutTime.get("partition")).isEqualTo(PARTITION); assertThat(result.get("offset")).isEqualTo(OFFSET); assertThat(resultWithoutTime.get("offset")).isEqualTo(OFFSET); - assertThat(result.get("timestamp")).isEqualTo(TIMESTAMP); - assertThat(resultWithoutTime.get("timestamp")).isNull(); + assertThat(result.get("record_timestamp")).isEqualTo(TIMESTAMP); + assertThat(resultWithoutTime.get("record_timestamp")).isNull(); } @Test @DisplayName("appendToMap should append record metadata under the configured key") public void appendToMap() { - KafkaMetadataAppender appenderWithExternalField = - new KafkaMetadataAppender( - EXT_FIELD, SCHEMA_WITH_EXT_FIELD, KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); + RecordAppender appenderWithExternalField = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, true, + KafkaMetadataAppender.KEY_METADATA_IS_NESTED, true, + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, "external,value")); SinkRecord recordWithTimestamp = genSinkRecordAsMap(true); SinkRecord recordWithoutTimestamp = genSinkRecordAsMap(false); @@ -166,7 +172,7 @@ public void appendToMap() { "topic", TOPIC, "partition", PARTITION, "offset", OFFSET, - "timestamp", TIMESTAMP, + "record_timestamp", TIMESTAMP, "external", "value")); Map expectedWithoutTime = @@ -180,10 +186,9 @@ public void appendToMap() { "offset", OFFSET, "external", "value")); - Map result = - appenderWithExternalField.appendToMap(recordWithTimestamp, record1); + Map result = appenderWithExternalField.addToMap(recordWithTimestamp, record1); Map resultWithoutTime = - appenderWithExternalField.appendToMap(recordWithoutTimestamp, record2); + appenderWithExternalField.addToMap(recordWithoutTimestamp, record2); assertThat(result).isEqualTo(expected); assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); From fef04f8b1ea4d88cda9515dad0dece8d95cb3df6 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 10 Mar 2024 21:21:00 -0700 Subject: [PATCH 08/17] readme fixes --- kafka-connect-transforms/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index 0ac68d71..cef0d356 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -48,7 +48,7 @@ It will promote the `before` or `after` element fields to top level and add the `_cdc.op`, `_cdc.ts`, `_cdc.offset`, `_cdc.source`, `_cdc.target`, and `_cdc.key`. If `cdc.kafka.include_metadata` it will add the following metadata fields: -`_kafka_metadata_topic`, `kafka_metadata_partition`, `kafka_metadata_offset`, and `kafka_metadata_record_timestamp`. +`_kafka_metadata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, and `_kafka_metadata_record_timestamp`. ## Configuration @@ -61,4 +61,4 @@ If `cdc.kafka.include_metadata` it will add the following metadata fields: | cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | If `cdc.kafka.include_metadata` it will add the following metadata fields: -`_kafka_metadata_topic`, `kafka_metadata_partition`, `kafka_metadata_offset`, and `kafka_metadata_record_timestamp`. \ No newline at end of file +`_kafka_metadata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, and `_kafka_metadata_record_timestamp`. \ No newline at end of file From ea9efe6343b7e9d17d8b7207f4a0e920068cbd7a Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 11 Mar 2024 10:10:15 -0700 Subject: [PATCH 09/17] fin --- kafka-connect-transforms/README.md | 55 +++++-- .../connect/transforms/DebeziumTransform.java | 26 ++- .../connect/transforms/DmsTransform.java | 27 +++- .../transforms/KafkaMetadataTransform.java | 132 +++++++++++++++ .../util/KafkaMetadataAppender.java | 30 ++-- .../KafkaMetadataTransformTest.java | 150 ++++++++++++++++++ .../util/KafkaMetadataAppenderTest.java | 89 +++++++++++ 7 files changed, 479 insertions(+), 30 deletions(-) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransform.java create mode 100644 kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransformTest.java diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index cef0d356..b1c9c47f 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -24,6 +24,27 @@ The `CopyValue` SMT copies a value from one field to a new field. "transforms.copyId.target.field": "id_copy", ``` +# KafkaMetadataTransform +_(Experimental)_ + +The `KafkaMetadata` injects `topic`, `partition`, `offset`, `record timestamp`. + +## Configuration + +| Property | Description (default value) | +|--------------------------------------------|-----------------------------------------------------------------------------------| +| transforms.kafka_metadata.include_metadata | (true) includes kafka metadata. False becomes a no-op | +| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | +| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| transforms.kafka_metadata.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.record_timestamp` + +If `nested` is off: +`_kafka_metdata_topic`, `kafka_metadata_partition`, `kafka_metadata_offset`, `kafka_metadata_record_timestamp` + # DmsTransform _(Experimental)_ @@ -33,12 +54,16 @@ It will promote the `data` element fields to top level and add the following met ## Configuration -| Property | Description | -|:-----------------------------|:--------------------------------------------------------------------------------------------------------| -| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | -| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | -| cdc.kafka.metadata_is_nested | If true nests metadata on a struct under field name, otherwise prefixes metadata fields with field name | -| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | +The DMS transform can also append Kafka Metadata without an additional record copy as per the `KafkaMetadataTransform` with the following +configuration: + +| Property | Description (default value) | +|--------------------------------------------|-----------------------------------------------------------------------------------| +| transforms.kafka_metadata.include_metadata | (false) includes kafka metadata. False will not append data to DMS transform | +| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | +| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| transforms.kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | + # DebeziumTransform _(Experimental)_ @@ -47,18 +72,18 @@ The `DebeziumTransform` SMT transforms a Debezium formatted message for use by t It will promote the `before` or `after` element fields to top level and add the following metadata fields: `_cdc.op`, `_cdc.ts`, `_cdc.offset`, `_cdc.source`, `_cdc.target`, and `_cdc.key`. -If `cdc.kafka.include_metadata` it will add the following metadata fields: -`_kafka_metadata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, and `_kafka_metadata_record_timestamp`. - ## Configuration | Property | Description | |:-----------------------------|:--------------------------------------------------------------------------------------------------------| | cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | -| cdc.kafka.include_metadata | Boolean (true) to append Kafka topic/partition/offset/timestamp metadta to each record | -| cdc.kafka.metadata_field | Key to append metadata on. Defaults to `_kafka_metadta` | -| cdc.kafka.metadata_is_nested | If true nests metadata on a struct under field name, otherwise prefixes metadata fields with field name | -| cdc.kafka.external_field | Optional `key,value` string to append a static field with Kafka Metadata. E.g. cluster name | -If `cdc.kafka.include_metadata` it will add the following metadata fields: -`_kafka_metadata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, and `_kafka_metadata_record_timestamp`. \ No newline at end of file +The Debezium transform can also append Kafka Metadata without an additional record copy as per the `KafkaMetadataTransform` with the following +configuration: + +| Property | Description (default value) | +|--------------------------------------------|-----------------------------------------------------------------------------------| +| transforms.kafka_metadata.include_metadata | (false) includes kafka metadata. False will not append data to DMS transform | +| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | +| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| transforms.kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | \ No newline at end of file diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index 4d80cc50..ad831d92 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -54,7 +54,31 @@ public class DebeziumTransform> implements Transforma ConfigDef.Type.STRING, null, Importance.MEDIUM, - "Pattern to use for setting the CDC target field value."); + "Pattern to use for setting the CDC target field value.") + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.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( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); private String cdcTargetPattern; diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index fe4132d5..25f148da 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -32,7 +32,32 @@ public class DmsTransform> implements Transformation { private static final Logger LOG = LoggerFactory.getLogger(DmsTransform.class.getName()); - public static final ConfigDef CONFIG_DEF = new ConfigDef(); + public static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.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( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); private RecordAppender kafkaAppender = null; @Override 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..f098e365 --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransform.java @@ -0,0 +1,132 @@ +/* + * 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 io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; +import io.tabular.iceberg.connect.transforms.util.RecordAppender; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +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.transforms.Transformation; +import org.apache.kafka.connect.transforms.util.Requirements; +import org.apache.kafka.connect.transforms.util.SchemaUtil; + +public class KafkaMetadataTransform> implements Transformation { + private RecordAppender kafkaAppender; + + public static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + ConfigDef.Type.BOOLEAN, + false, + ConfigDef.Importance.LOW, + "Include appending of Kafka metadata to SinkRecord") + .define( + KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + ConfigDef.Type.STRING, + KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + ConfigDef.Importance.LOW, + "field to append Kafka metadata under") + .define( + KafkaMetadataAppender.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( + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + ConfigDef.Type.STRING, + "none", + ConfigDef.Importance.LOW, + "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + + @Override + public R apply(R record) { + if (record.value() == null) { + return record; + } else if (record.valueSchema() == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applyWithSchema(R record) { + Struct value = Requirements.requireStruct(record.value(), "KafkaMetadata transform"); + Schema newSchema = makeUpdatedSchema(record.valueSchema()); + Struct newValue = new Struct(newSchema); + for (Field field : record.valueSchema().fields()) { + newValue.put(field.name(), value.get(field)); + } + kafkaAppender.addToStruct(record, newValue); + return record.newRecord( + record.topic(), + record.kafkaPartition(), + record.keySchema(), + record.key(), + newSchema, + newValue, + record.timestamp(), + record.headers()); + } + + private R applySchemaless(R record) { + Map value = Requirements.requireMap(record.value(), "KafkaMetadata transform"); + Map newValue = Maps.newHashMap(value); + kafkaAppender.addToMap(record, newValue); + + return record.newRecord( + record.topic(), + record.kafkaPartition(), + record.keySchema(), + record.key(), + null, + 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()); + } + kafkaAppender.addToSchema(builder); + return builder.build(); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() {} + + @Override + public void configure(Map configs) { + kafkaAppender = KafkaMetadataAppender.from(configs); + } +} diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java index 5b8ad0d8..c7075d53 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -32,10 +32,10 @@ public abstract class KafkaMetadataAppender { - public static final String INCLUDE_KAFKA_METADATA = "cdc.kafka.include_metadata"; - public static final String EXTERNAL_KAFKA_METADATA = "cdc.kafka.external_field"; - public static final String KEY_METADATA_FIELD_NAME = "cdc.kafka.metadata_field"; - public static final String KEY_METADATA_IS_NESTED = "cdc.kafka.metadata_is_nested"; + public static final String INCLUDE_KAFKA_METADATA = "transforms.kafka_metadata.include_metadata"; + public static final String EXTERNAL_KAFKA_METADATA = "transforms.kafka_metadata.external_field"; + public static final String KEY_METADATA_FIELD_NAME = "transforms.kafka_metadata.metadata_field"; + public static final String KEY_METADATA_IS_NESTED = "transforms.kafka_metadata.nested"; public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; public static final ConfigDef CONFIG_DEF = @@ -43,7 +43,7 @@ public abstract class KafkaMetadataAppender { .define( KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, ConfigDef.Type.BOOLEAN, - false, + true, ConfigDef.Importance.LOW, "Include appending of Kafka metadata to SinkRecord") .define( @@ -75,8 +75,7 @@ public static > RecordAppender from(Map } private static > RecordAppender from(SimpleConfig config) { - RecordAppender externalFieldAppender = - getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA)); + RecordAppender externalFieldAppender; String metadataFieldName = config.getString(KEY_METADATA_FIELD_NAME); Boolean nestedMetadata = config.getBoolean(KEY_METADATA_IS_NESTED); @@ -91,6 +90,9 @@ private static > RecordAppender from(SimpleConfig offsetFieldName = "offset"; timestampFieldName = "record_timestamp"; + externalFieldAppender = + getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA), name -> name); + SchemaBuilder nestedSchemaBuilder = SchemaBuilder.struct(); nestedSchemaBuilder .field(topicFieldName, Schema.STRING_SCHEMA) @@ -149,6 +151,8 @@ public Map addToMap(R record, Map map) { offsetFieldName = namer.apply("offset"); timestampFieldName = namer.apply("record_timestamp"); + externalFieldAppender = + getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA), namer); return new RecordAppender() { @Override public SchemaBuilder addToSchema(SchemaBuilder builder) { @@ -177,14 +181,14 @@ public Struct addToStruct(R record, Struct struct) { @Override public Map addToMap(R record, Map map) { - map.put("topic", record.topic()); - map.put("partition", record.kafkaPartition()); + map.put(topicFieldName, record.topic()); + map.put(partitionFieldName, record.kafkaPartition()); if (record instanceof SinkRecord) { SinkRecord sinkRecord = (SinkRecord) record; - map.put("offset", sinkRecord.kafkaOffset()); + map.put(offsetFieldName, sinkRecord.kafkaOffset()); } if (record.timestamp() != null) { - map.put("record_timestamp", record.timestamp()); + map.put(timestampFieldName, record.timestamp()); } externalFieldAppender.addToMap(record, map); return map; @@ -212,7 +216,7 @@ public Map addToMap(R record, Map map) { } private static > RecordAppender getExternalFieldAppender( - String field) { + String field, Function fieldNamer) { if (field.equals("none")) { return new NoOpRecordAppender<>(); } @@ -221,7 +225,7 @@ private static > RecordAppender getExternalFieldAp throw new ConfigException( String.format("Could not parse %s for %s", field, EXTERNAL_KAFKA_METADATA)); } - String fieldName = parts[0]; + String fieldName = fieldNamer.apply(parts[0]); String fieldValue = parts[1]; return new RecordAppender() { 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..5b8bd592 --- /dev/null +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/KafkaMetadataTransformTest.java @@ -0,0 +1,150 @@ +/* + * 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()); + 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_record_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") + 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_record_timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java index 8ce8641a..3b7d1e2e 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppenderTest.java @@ -146,6 +146,40 @@ public void appendToStruct() { assertThat(resultWithoutTime.get("record_timestamp")).isNull(); } + @Test + @DisplayName("appendToStruct should append flattened record metadata under the configured key") + public void appendToStructFlat() { + RecordAppender appenderWithExternalField = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true, + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + "external,value")); + + SinkRecord recordWithTimestamp = genSinkRecord(true); + SinkRecord recordWithoutTimestamp = genSinkRecord(false); + + Schema schema = + appenderWithExternalField.addToSchema( + SchemaBuilder.struct().field("id", Schema.STRING_SCHEMA)); + + Struct result = appenderWithExternalField.addToStruct(recordWithTimestamp, new Struct(schema)); + Struct resultWithoutTime = + appenderWithExternalField.addToStruct(recordWithoutTimestamp, new Struct(schema)); + + assertThat(result.get("_kafka_metadata_external")).isEqualTo("value"); + assertThat(resultWithoutTime.get("_kafka_metadata_external")).isEqualTo("value"); + assertThat(result.get("_kafka_metadata_topic")).isEqualTo(TOPIC); + assertThat(resultWithoutTime.get("_kafka_metadata_topic")).isEqualTo(TOPIC); + assertThat(result.get("_kafka_metadata_partition")).isEqualTo(PARTITION); + assertThat(resultWithoutTime.get("_kafka_metadata_partition")).isEqualTo(PARTITION); + assertThat(result.get("_kafka_metadata_offset")).isEqualTo(OFFSET); + assertThat(resultWithoutTime.get("_kafka_metadata_offset")).isEqualTo(OFFSET); + assertThat(result.get("_kafka_metadata_record_timestamp")).isEqualTo(TIMESTAMP); + assertThat(resultWithoutTime.get("_kafka_metadata_record_timestamp")).isNull(); + } + @Test @DisplayName("appendToMap should append record metadata under the configured key") public void appendToMap() { @@ -193,4 +227,59 @@ public void appendToMap() { assertThat(result).isEqualTo(expected); assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); } + + @Test + @DisplayName("appendToMap should append flattened record metadata under the configured key") + public void appendToMapFlat() { + RecordAppender appenderWithExternalField = + KafkaMetadataAppender.from( + ImmutableMap.of( + KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, + true, + KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + "external,value")); + + SinkRecord recordWithTimestamp = genSinkRecordAsMap(true); + SinkRecord recordWithoutTimestamp = genSinkRecordAsMap(false); + + Map record1 = Maps.newHashMap(); + record1.put("id", "id"); + + Map record2 = Maps.newHashMap(); + record2.put("id", "id"); + Map expected = + ImmutableMap.of( + "id", + "id", + "_kafka_metadata_topic", + TOPIC, + "_kafka_metadata_partition", + PARTITION, + "_kafka_metadata_offset", + OFFSET, + "_kafka_metadata_record_timestamp", + TIMESTAMP, + "_kafka_metadata_external", + "value"); + + Map expectedWithoutTime = + ImmutableMap.of( + "id", + "id", + "_kafka_metadata_topic", + TOPIC, + "_kafka_metadata_partition", + PARTITION, + "_kafka_metadata_offset", + OFFSET, + "_kafka_metadata_external", + "value"); + + Map result = appenderWithExternalField.addToMap(recordWithTimestamp, record1); + Map resultWithoutTime = + appenderWithExternalField.addToMap(recordWithoutTimestamp, record2); + + assertThat(result).isEqualTo(expected); + assertThat(resultWithoutTime).isEqualTo(expectedWithoutTime); + } } From 52cbda97595df8b2b3e235bea534023745cd4826 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 27 Mar 2024 14:28:44 -0700 Subject: [PATCH 10/17] fix keys --- kafka-connect-transforms/README.md | 38 +++++++++---------- .../util/KafkaMetadataAppender.java | 8 ++-- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index b1c9c47f..32d94006 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -31,12 +31,12 @@ The `KafkaMetadata` injects `topic`, `partition`, `offset`, `record timestamp`. ## Configuration -| Property | Description (default value) | -|--------------------------------------------|-----------------------------------------------------------------------------------| -| transforms.kafka_metadata.include_metadata | (true) includes kafka metadata. False becomes a no-op | -| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | -| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | -| transforms.kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | +| Property | Description (default value) | +|-------------------------------|-----------------------------------------------------------------------------------| +| kafka_metadata.include | (true) includes kafka metadata. False becomes a no-op | +| kafka_metadata.field_name | (_kafka_metadata) prefix for fields | +| kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | If `nested` is on: @@ -55,14 +55,14 @@ It will promote the `data` element fields to top level and add the following met ## Configuration The DMS transform can also append Kafka Metadata without an additional record copy as per the `KafkaMetadataTransform` with the following -configuration: +configuration: -| Property | Description (default value) | -|--------------------------------------------|-----------------------------------------------------------------------------------| -| transforms.kafka_metadata.include_metadata | (false) includes kafka metadata. False will not append data to DMS transform | -| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | -| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | -| transforms.kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | +| Property | Description (default value) | +|-------------------------------|-----------------------------------------------------------------------------------| +| kafka_metadata.include | (true) includes kafka metadata. False becomes a no-op | +| kafka_metadata.field_name | (_kafka_metadata) prefix for fields | +| kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | # DebeziumTransform @@ -81,9 +81,9 @@ It will promote the `before` or `after` element fields to top level and add the The Debezium transform can also append Kafka Metadata without an additional record copy as per the `KafkaMetadataTransform` with the following configuration: -| Property | Description (default value) | -|--------------------------------------------|-----------------------------------------------------------------------------------| -| transforms.kafka_metadata.include_metadata | (false) includes kafka metadata. False will not append data to DMS transform | -| transforms.kafka_metadata.metadata_field | (_kafka_metadata) prefix for fields | -| transforms.kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | -| transforms.kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | \ No newline at end of file +| Property | Description (default value) | +|-------------------------------|-----------------------------------------------------------------------------------| +| kafka_metadata.include | (true) includes kafka metadata. False becomes a no-op | +| kafka_metadata.field_name | (_kafka_metadata) prefix for fields | +| kafka_metadata.nested | (false) if true, nests data on a struct else adds to top level as prefixed fields | +| kafka_metadata.external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) | \ No newline at end of file diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java index c7075d53..03f165fb 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/util/KafkaMetadataAppender.java @@ -32,10 +32,10 @@ public abstract class KafkaMetadataAppender { - public static final String INCLUDE_KAFKA_METADATA = "transforms.kafka_metadata.include_metadata"; - public static final String EXTERNAL_KAFKA_METADATA = "transforms.kafka_metadata.external_field"; - public static final String KEY_METADATA_FIELD_NAME = "transforms.kafka_metadata.metadata_field"; - public static final String KEY_METADATA_IS_NESTED = "transforms.kafka_metadata.nested"; + public static final String INCLUDE_KAFKA_METADATA = "kafka_metadata.include"; + public static final String EXTERNAL_KAFKA_METADATA = "kafka_metadata.external_field"; + public static final String KEY_METADATA_FIELD_NAME = "kafka.metadata.field_name"; + public static final String KEY_METADATA_IS_NESTED = "kafka_metadata.nested"; public static final String DEFAULT_METADATA_FIELD_NAME = "_kafka_metadata"; public static final ConfigDef CONFIG_DEF = From 1d3ecf62180dfc06f6bbae5dc1102b47d74567cc Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 09:26:08 -0600 Subject: [PATCH 11/17] comments, removing cdc/dms options --- .../connect/transforms/DebeziumTransform.java | 39 +-- .../connect/transforms/DmsTransform.java | 38 +-- .../transforms/KafkaMetadataTransform.java | 244 +++++++++++++++--- .../transforms/DebeziumTransformTest.java | 61 ----- .../connect/transforms/DmsTransformTest.java | 27 -- .../KafkaMetadataTransformTest.java | 118 ++++++++- 6 files changed, 326 insertions(+), 201 deletions(-) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java index ad831d92..1e79e211 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DebeziumTransform.java @@ -18,8 +18,6 @@ */ package io.tabular.iceberg.connect.transforms; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; -import io.tabular.iceberg.connect.transforms.util.RecordAppender; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; @@ -45,7 +43,6 @@ public class DebeziumTransform> implements Transforma private static final String CDC_TARGET_PATTERN = "cdc.target.pattern"; private static final String DB_PLACEHOLDER = "{db}"; private static final String TABLE_PLACEHOLDER = "{table}"; - private RecordAppender kafkaAppender; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -54,31 +51,7 @@ public class DebeziumTransform> implements Transforma ConfigDef.Type.STRING, null, Importance.MEDIUM, - "Pattern to use for setting the CDC target field value.") - .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - ConfigDef.Importance.LOW, - "Include appending of Kafka metadata to SinkRecord") - .define( - KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - ConfigDef.Importance.LOW, - "field to append Kafka metadata under") - .define( - KafkaMetadataAppender.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( - KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - ConfigDef.Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); + "Pattern to use for setting the CDC target field value."); private String cdcTargetPattern; @@ -86,7 +59,6 @@ public class DebeziumTransform> implements Transforma public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); cdcTargetPattern = config.getString(CDC_TARGET_PATTERN); - kafkaAppender = KafkaMetadataAppender.from(props); } @Override @@ -138,8 +110,6 @@ private R applyWithSchema(R record) { } newValue.put(CdcConstants.COL_CDC, cdcMetadata); - kafkaAppender.addToStruct(record, newValue); - return record.newRecord( record.topic(), record.kafkaPartition(), @@ -168,9 +138,6 @@ private R applySchemaless(R record) { return null; } - // create the new value - Map newValue = Maps.newHashMap((Map) payload); - // create the CDC metadata Map cdcMetadata = Maps.newHashMap(); cdcMetadata.put(CdcConstants.COL_OP, op); @@ -178,13 +145,14 @@ private R applySchemaless(R record) { if (record instanceof SinkRecord) { cdcMetadata.put(CdcConstants.COL_OFFSET, ((SinkRecord) record).kafkaOffset()); } - kafkaAppender.addToMap(record, newValue); setTableAndTargetFromSourceMap(value.get("source"), cdcMetadata); if (record.key() instanceof Map) { cdcMetadata.put(CdcConstants.COL_KEY, record.key()); } + // create the new value + Map newValue = Maps.newHashMap((Map) payload); newValue.put(CdcConstants.COL_CDC, cdcMetadata); return record.newRecord( @@ -269,7 +237,6 @@ private Schema makeUpdatedSchema(Schema schema, Schema cdcSchema) { } builder.field(CdcConstants.COL_CDC, cdcSchema); - kafkaAppender.addToSchema(builder); return builder.build(); } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java index 25f148da..4db4bb3a 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DmsTransform.java @@ -18,8 +18,6 @@ */ package io.tabular.iceberg.connect.transforms; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; -import io.tabular.iceberg.connect.transforms.util.RecordAppender; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; @@ -32,33 +30,7 @@ public class DmsTransform> implements Transformation { private static final Logger LOG = LoggerFactory.getLogger(DmsTransform.class.getName()); - public static final ConfigDef CONFIG_DEF = - new ConfigDef() - .define( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - ConfigDef.Importance.LOW, - "Include appending of Kafka metadata to SinkRecord") - .define( - KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, - ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, - ConfigDef.Importance.LOW, - "field to append Kafka metadata under") - .define( - KafkaMetadataAppender.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( - KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, - ConfigDef.Type.STRING, - "none", - ConfigDef.Importance.LOW, - "key,value representing a String to be injected on Kafka metadata (e.g. Cluster)"); - private RecordAppender kafkaAppender = null; + private static final ConfigDef EMPTY_CONFIG = new ConfigDef(); @Override public R apply(R record) { @@ -110,8 +82,6 @@ private R applySchemaless(R record) { Map newValue = Maps.newHashMap((Map) dataObj); newValue.put(CdcConstants.COL_CDC, cdcMetadata); - this.kafkaAppender.addToMap(record, newValue); - return record.newRecord( record.topic(), record.kafkaPartition(), @@ -124,14 +94,12 @@ private R applySchemaless(R record) { @Override public ConfigDef config() { - return CONFIG_DEF; + return EMPTY_CONFIG; } @Override public void close() {} @Override - public void configure(Map configs) { - this.kafkaAppender = KafkaMetadataAppender.from(configs); - } + public void configure(Map configs) {} } 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 index f098e365..6ea12fc0 100644 --- 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 @@ -18,52 +18,230 @@ */ package io.tabular.iceberg.connect.transforms; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; -import io.tabular.iceberg.connect.transforms.util.RecordAppender; 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.connect.connector.ConnectRecord; +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 RecordAppender kafkaAppender; +public class KafkaMetadataTransform implements Transformation { - public static final ConfigDef CONFIG_DEF = + private interface RecordAppender { + + SchemaBuilder addToSchema(SchemaBuilder builder); + + Struct addToStruct(SinkRecord record, Struct struct); + + Map addToMap(SinkRecord record, Map map); + } + + private static class NoOpRecordAppender implements RecordAppender { + + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder; + } + + @Override + public Struct addToStruct(SinkRecord record, Struct struct) { + return struct; + } + + @Override + public Map addToMap(SinkRecord record, Map map) { + return 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 SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(fieldName, Schema.STRING_SCHEMA); + } + + @Override + public Struct addToStruct(SinkRecord record, Struct struct) { + return struct.put(fieldName, fieldValue); + } + + @Override + public Map addToMap(SinkRecord record, Map map) { + map.put(fieldName, fieldValue); + return map; + } + }; + } + + private static final String TOPIC = "topic"; + private static final String PARTITION = "partition"; + + private static final String OFFSET = "offset"; + + private static final String TIMESTAMP = "record_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( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - ConfigDef.Type.BOOLEAN, - false, - ConfigDef.Importance.LOW, - "Include appending of Kafka metadata to SinkRecord") - .define( - KafkaMetadataAppender.KEY_METADATA_FIELD_NAME, + KEY_METADATA_FIELD_NAME, ConfigDef.Type.STRING, - KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME, + DEFAULT_METADATA_FIELD_NAME, ConfigDef.Importance.LOW, - "field to append Kafka metadata under") + "the field to append Kafka metadata under (or prefix fields with)") .define( - KafkaMetadataAppender.KEY_METADATA_IS_NESTED, + 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( - KafkaMetadataAppender.EXTERNAL_KAFKA_METADATA, + EXTERNAL_KAFKA_METADATA, ConfigDef.Type.STRING, - "none", + 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.OPTIONAL_INT64_SCHEMA) + .field(TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA); + externalFieldAppender.addToSchema(nestedSchemaBuilder); + + Schema nestedSchema = nestedSchemaBuilder.build(); + + return new RecordAppender() { + @Override + public SchemaBuilder addToSchema(SchemaBuilder builder) { + return builder.field(metadataFieldName, nestedSchema); + } + + @Override + public Struct 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); + return struct; + } + + @Override + public Map 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); + return map; + } + }; + + } 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 SchemaBuilder addToSchema(SchemaBuilder builder) { + builder + .field(topicFieldName, Schema.STRING_SCHEMA) + .field(partitionFieldName, Schema.INT32_SCHEMA) + .field(offsetFieldName, Schema.OPTIONAL_INT64_SCHEMA) + .field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA); + return externalFieldAppender.addToSchema(builder); + } + + @Override + public Struct 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); + return struct; + } + + @Override + public Map 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); + return map; + } + }; + } + } + @Override - public R apply(R record) { + public SinkRecord apply(SinkRecord record) { if (record.value() == null) { return record; } else if (record.valueSchema() == null) { @@ -73,14 +251,14 @@ public R apply(R record) { } } - private R applyWithSchema(R record) { + private SinkRecord applyWithSchema(SinkRecord record) { Struct value = Requirements.requireStruct(record.value(), "KafkaMetadata transform"); Schema newSchema = makeUpdatedSchema(record.valueSchema()); Struct newValue = new Struct(newSchema); for (Field field : record.valueSchema().fields()) { newValue.put(field.name(), value.get(field)); } - kafkaAppender.addToStruct(record, newValue); + recordAppender.addToStruct(record, newValue); return record.newRecord( record.topic(), record.kafkaPartition(), @@ -92,10 +270,19 @@ private R applyWithSchema(R record) { record.headers()); } - private R applySchemaless(R record) { + 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); - kafkaAppender.addToMap(record, newValue); + recordAppender.addToMap(record, newValue); return record.newRecord( record.topic(), @@ -108,15 +295,6 @@ private R applySchemaless(R record) { 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()); - } - kafkaAppender.addToSchema(builder); - return builder.build(); - } - @Override public ConfigDef config() { return CONFIG_DEF; @@ -127,6 +305,6 @@ public void close() {} @Override public void configure(Map configs) { - kafkaAppender = KafkaMetadataAppender.from(configs); + recordAppender = getRecordAppender(configs); } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java index f652819c..c9f34848 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DebeziumTransformTest.java @@ -19,9 +19,7 @@ package io.tabular.iceberg.connect.transforms; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.math.BigDecimal; import java.time.Instant; import java.util.Map; @@ -30,7 +28,6 @@ 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.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.Test; @@ -92,35 +89,6 @@ public void testDebeziumTransformSchemaless() { assertThat(cdcMetadata.get("source")).isEqualTo("schema.tbl"); assertThat(cdcMetadata.get("target")).isEqualTo("schema_x.tbl_x"); assertThat(cdcMetadata.get("key")).isInstanceOf(Map.class); - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); - } - } - - @Test - @SuppressWarnings("unchecked") - public void testDebeziumTransformSchemalessAndKafkaMetadata() { - try (DebeziumTransform smt = new DebeziumTransform<>()) { - smt.configure( - ImmutableMap.of( - "cdc.target.pattern", - "{db}_x.{table}_x", - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - true, - KafkaMetadataAppender.KEY_METADATA_IS_NESTED, - true)); - - Map event = createDebeziumEventMap("u"); - Map key = ImmutableMap.of("account_id", 1L); - SinkRecord record = new SinkRecord("topic", 0, null, key, null, event, 0); - - SinkRecord result = smt.apply(record); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); - - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); - Map metadata = - (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - assertThat(metadata.get("topic")).isEqualTo("topic"); } } @@ -138,8 +106,6 @@ public void testDebeziumTransformWithSchema() { Struct value = (Struct) result.value(); assertThat(value.get("account_id")).isEqualTo(1L); - assertThrows( - DataException.class, () -> value.get(KafkaMetadataAppender.INCLUDE_KAFKA_METADATA)); Struct cdcMetadata = value.getStruct("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); @@ -149,33 +115,6 @@ public void testDebeziumTransformWithSchema() { } } - @Test - public void testDebeziumTransformWithSchemaAndKafkaMetadata() { - try (DebeziumTransform smt = new DebeziumTransform<>()) { - smt.configure( - ImmutableMap.of( - "cdc.target.pattern", - "{db}_x.{table}_x", - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - true, - KafkaMetadataAppender.KEY_METADATA_IS_NESTED, - true)); - - Struct event = createDebeziumEventStruct("u"); - Struct key = new Struct(KEY_SCHEMA).put("account_id", 1L); - SinkRecord record = new SinkRecord("topic", 0, KEY_SCHEMA, key, VALUE_SCHEMA, event, 0); - - SinkRecord result = smt.apply(record); - assertThat(result.value()).isInstanceOf(Struct.class); - Struct value = (Struct) result.value(); - - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)) - .isInstanceOf(Struct.class); - Struct metadata = (Struct) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - assertThat(metadata.get("topic")).isEqualTo(record.topic()); - } - } - private Map createDebeziumEventMap(String operation) { Map source = ImmutableMap.of( diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java index 79900da8..6bb2f20e 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/DmsTransformTest.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import io.tabular.iceberg.connect.transforms.util.KafkaMetadataAppender; import java.time.Instant; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -33,7 +32,6 @@ public class DmsTransformTest { @SuppressWarnings("unchecked") public void testDmsTransform() { try (DmsTransform smt = new DmsTransform<>()) { - smt.configure(ImmutableMap.of()); Map event = createDmsEvent("update"); SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); @@ -46,31 +44,6 @@ public void testDmsTransform() { Map cdcMetadata = (Map) value.get("_cdc"); assertThat(cdcMetadata.get("op")).isEqualTo("U"); assertThat(cdcMetadata.get("source")).isEqualTo("db.tbl"); - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNull(); - } - } - - @Test - public void testDmsTransformWithkafkaMetadata() { - try (DmsTransform smt = new DmsTransform<>()) { - smt.configure( - ImmutableMap.of( - KafkaMetadataAppender.INCLUDE_KAFKA_METADATA, - true, - KafkaMetadataAppender.KEY_METADATA_IS_NESTED, - true)); - - Map event = createDmsEvent("update"); - SinkRecord record = new SinkRecord("topic", 0, null, null, null, event, 0); - - SinkRecord result = smt.apply(record); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); - - assertThat(value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME)).isNotNull(); - Map metadata = - (Map) value.get(KafkaMetadataAppender.DEFAULT_METADATA_FIELD_NAME); - assertThat(metadata.get("topic")).isEqualTo("topic"); } } 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 index 5b8bd592..2a2ffccf 100644 --- 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 @@ -49,7 +49,7 @@ public void testNullRecord() { SinkRecord record = new SinkRecord( TOPIC, PARTITION, null, null, null, null, OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - try (KafkaMetadataTransform smt = new KafkaMetadataTransform<>()) { + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { smt.configure(ImmutableMap.of()); SinkRecord result = smt.apply(record); assertThat(record).isSameAs(result); @@ -81,7 +81,7 @@ public void testThrowIfNotExpectedValue() { OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - try (KafkaMetadataTransform smt = new KafkaMetadataTransform<>()) { + try (KafkaMetadataTransform smt = new KafkaMetadataTransform()) { smt.configure(ImmutableMap.of()); assertThrows(RuntimeException.class, () -> smt.apply(recordNotMap)); assertThrows(RuntimeException.class, () -> smt.apply(recordNotStruct)); @@ -102,8 +102,69 @@ public void testAppendsToStucts() { OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - try (KafkaMetadataTransform smt = new KafkaMetadataTransform<>()) { - smt.configure(ImmutableMap.of()); + 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_record_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("record_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(); @@ -112,9 +173,17 @@ public void testAppendsToStucts() { assertThat(value.get("_kafka_metadata_partition")).isEqualTo(result.kafkaPartition()); assertThat(value.get("_kafka_metadata_offset")).isEqualTo(result.kafkaOffset()); assertThat(value.get("_kafka_metadata_record_timestamp")).isEqualTo(result.timestamp()); - assertThat(result.timestampType()).isEqualTo(record.timestampType()); - assertThat(result.key()).isEqualTo(record.key()); - assertThat(result.keySchema()).isEqualTo(record.keySchema()); + 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"))); } } @@ -132,11 +201,11 @@ public void testAppendToMaps() { OFFSET, TIMESTAMP, TimestampType.CREATE_TIME); - try (KafkaMetadataTransform smt = new KafkaMetadataTransform<>()) { + 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(); + 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()); @@ -147,4 +216,35 @@ public void testAppendToMaps() { 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("record_timestamp")).isEqualTo(result.timestamp()); + assertThat(result.timestampType()).isEqualTo(record.timestampType()); + assertThat(result.key()).isEqualTo(record.key()); + assertThat(result.keySchema()).isEqualTo(record.keySchema()); + } + } } From 50a27dd1c8789f46ca06d12bca81453b81596964 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 09:33:01 -0600 Subject: [PATCH 12/17] timestamp --- .../connect/transforms/KafkaMetadataTransform.java | 2 +- .../connect/transforms/KafkaMetadataTransformTest.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) 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 index 6ea12fc0..14b9c6fd 100644 --- 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 @@ -99,7 +99,7 @@ public Map addToMap(SinkRecord record, Map map) private static final String OFFSET = "offset"; - private static final String TIMESTAMP = "record_timestamp"; + 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"; 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 index 2a2ffccf..29e1acec 100644 --- 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 @@ -111,7 +111,7 @@ public void testAppendsToStucts() { 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_record_timestamp")).isEqualTo(result.timestamp()); + 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()); @@ -142,7 +142,7 @@ public void testAppendsToStructsNested() { assertThat(metadata.get("topic")).isEqualTo(result.topic()); assertThat(metadata.get("partition")).isEqualTo(result.kafkaPartition()); assertThat(metadata.get("offset")).isEqualTo(result.kafkaOffset()); - assertThat(metadata.get("record_timestamp")).isEqualTo(result.timestamp()); + 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()); @@ -172,7 +172,7 @@ public void testAppendsToStuctsExternal() { 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_record_timestamp")).isEqualTo(result.timestamp()); + assertThat(value.get("_kafka_metadata_timestamp")).isEqualTo(result.timestamp()); assertThat(value.get("_kafka_metadata_external")).isEqualTo("value"); } } @@ -210,7 +210,7 @@ public void testAppendToMaps() { 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_record_timestamp")).isEqualTo(result.timestamp()); + 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()); @@ -241,7 +241,7 @@ public void testAppendToMapsNested() { assertThat(metadata.get("topic")).isEqualTo(result.topic()); assertThat(metadata.get("partition")).isEqualTo(result.kafkaPartition()); assertThat(metadata.get("offset")).isEqualTo(result.kafkaOffset()); - assertThat(metadata.get("record_timestamp")).isEqualTo(result.timestamp()); + 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()); From 84c8e9161c33707ef2a9db8ee340ff17ab994a2c Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 15:50:43 -0600 Subject: [PATCH 13/17] comments --- kafka-connect-transforms/README.md | 75 +++++++++++++++++++ .../transforms/KafkaMetadataTransform.java | 7 +- 2 files changed, 77 insertions(+), 5 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index 65591aaa..43ff793e 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -67,3 +67,78 @@ It will promote the `before` or `after` element fields to top level and add the | Property | Description | |---------------------|-----------------------------------------------------------------------------------| | cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` | + + +# JsonToMapTransform +_(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. + +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. + +Note: + +- You must use the `stringConverter` as the `value.converter` setting for your connector, not `jsonConverter` + - 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 + +| Property | Description (default value) | +|-------------|------------------------------------------| +| 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`. + +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 +json arrays have mixed types in which case they are converted to arrays of strings. + +Example json: + +```json +{ + "key": 1, + "array": [1,"two",3], + "empty_obj": {}, + "nested_obj": {"some_key": ["one", "two"]} +} +``` + +Will become the following if `json.root` is true: + +``` +SinkRecord.schema: + "payload" : (Optional) Map + +Sinkrecord.value (Struct): + "payload" : Map( + "key" : "1", + "array" : "[1,"two",3]" + "empty_obj": "{}" + "nested_obj": "{"some_key":["one","two"]}}" + ) +``` + +Will become the following if `json.root` is false + +``` +SinkRecord.schema: + "key": (Optional) Int32, + "array": (Optional) Array, + "nested_object": (Optional) Map + +SinkRecord.value (Struct): + "key" 1, + "array" ["1", "two", "3"] + "nested_object" Map ("some_key" : "["one", "two"]") +``` \ No newline at end of file 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 index 14b9c6fd..23aae5c1 100644 --- 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 @@ -96,11 +96,8 @@ public Map addToMap(SinkRecord record, Map map) 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"; @@ -152,7 +149,7 @@ private static RecordAppender getRecordAppender(SimpleConfig config) { nestedSchemaBuilder .field(TOPIC, Schema.STRING_SCHEMA) .field(PARTITION, Schema.INT32_SCHEMA) - .field(OFFSET, Schema.OPTIONAL_INT64_SCHEMA) + .field(OFFSET, Schema.INT64_SCHEMA) .field(TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA); externalFieldAppender.addToSchema(nestedSchemaBuilder); @@ -252,7 +249,7 @@ public SinkRecord apply(SinkRecord record) { } private SinkRecord applyWithSchema(SinkRecord record) { - Struct value = Requirements.requireStruct(record.value(), "KafkaMetadata transform"); + Struct value = Requirements.requireStruct(record.value(), "KafkaMetadataTransform"); Schema newSchema = makeUpdatedSchema(record.valueSchema()); Struct newValue = new Struct(newSchema); for (Field field : record.valueSchema().fields()) { From 73ecdfe522ae3c6a5915eb053aa0aaacf882ecc1 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 16:00:15 -0600 Subject: [PATCH 14/17] void signature --- .../transforms/KafkaMetadataTransform.java | 49 +++++++------------ 1 file changed, 18 insertions(+), 31 deletions(-) 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 index 23aae5c1..743aa000 100644 --- 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 @@ -37,29 +37,23 @@ public class KafkaMetadataTransform implements Transformation { private interface RecordAppender { - SchemaBuilder addToSchema(SchemaBuilder builder); + void addToSchema(SchemaBuilder builder); - Struct addToStruct(SinkRecord record, Struct struct); + void addToStruct(SinkRecord record, Struct struct); - Map addToMap(SinkRecord record, Map map); + void addToMap(SinkRecord record, Map map); } private static class NoOpRecordAppender implements RecordAppender { @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder; - } + public void addToSchema(SchemaBuilder builder) {} @Override - public Struct addToStruct(SinkRecord record, Struct struct) { - return struct; - } + public void addToStruct(SinkRecord record, Struct struct) {} @Override - public Map addToMap(SinkRecord record, Map map) { - return map; - } + public void addToMap(SinkRecord record, Map map) {} } private static RecordAppender getExternalFieldAppender( @@ -77,19 +71,18 @@ private static RecordAppender getExternalFieldAppender( return new RecordAppender() { @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder.field(fieldName, Schema.STRING_SCHEMA); + public void addToSchema(SchemaBuilder builder) { + builder.field(fieldName, Schema.STRING_SCHEMA); } @Override - public Struct addToStruct(SinkRecord record, Struct struct) { - return struct.put(fieldName, fieldValue); + public void addToStruct(SinkRecord record, Struct struct) { + struct.put(fieldName, fieldValue); } @Override - public Map addToMap(SinkRecord record, Map map) { + public void addToMap(SinkRecord record, Map map) { map.put(fieldName, fieldValue); - return map; } }; } @@ -102,7 +95,6 @@ public Map addToMap(SinkRecord record, Map map) 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 = @@ -157,12 +149,12 @@ private static RecordAppender getRecordAppender(SimpleConfig config) { return new RecordAppender() { @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { - return builder.field(metadataFieldName, nestedSchema); + public void addToSchema(SchemaBuilder builder) { + builder.field(metadataFieldName, nestedSchema); } @Override - public Struct addToStruct(SinkRecord record, Struct struct) { + public void addToStruct(SinkRecord record, Struct struct) { Struct nested = new Struct(nestedSchema); nested.put(TOPIC, record.topic()); nested.put(PARTITION, record.kafkaPartition()); @@ -172,11 +164,10 @@ public Struct addToStruct(SinkRecord record, Struct struct) { } externalFieldAppender.addToStruct(record, nested); struct.put(metadataFieldName, nested); - return struct; } @Override - public Map addToMap(SinkRecord record, Map map) { + public void addToMap(SinkRecord record, Map map) { Map nested = Maps.newHashMap(); nested.put(TOPIC, record.topic()); nested.put(PARTITION, record.kafkaPartition()); @@ -186,7 +177,6 @@ public Map addToMap(SinkRecord record, Map map) } externalFieldAppender.addToMap(record, nested); map.put(metadataFieldName, nested); - return map; } }; @@ -201,17 +191,16 @@ public Map addToMap(SinkRecord record, Map map) getExternalFieldAppender(config.getString(EXTERNAL_KAFKA_METADATA), namer); return new RecordAppender() { @Override - public SchemaBuilder addToSchema(SchemaBuilder builder) { + public void addToSchema(SchemaBuilder builder) { builder .field(topicFieldName, Schema.STRING_SCHEMA) .field(partitionFieldName, Schema.INT32_SCHEMA) .field(offsetFieldName, Schema.OPTIONAL_INT64_SCHEMA) .field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA); - return externalFieldAppender.addToSchema(builder); } @Override - public Struct addToStruct(SinkRecord record, Struct struct) { + public void addToStruct(SinkRecord record, Struct struct) { struct.put(topicFieldName, record.topic()); struct.put(partitionFieldName, record.kafkaPartition()); struct.put(offsetFieldName, record.kafkaOffset()); @@ -219,11 +208,10 @@ public Struct addToStruct(SinkRecord record, Struct struct) { struct.put(timestampFieldName, record.timestamp()); } externalFieldAppender.addToStruct(record, struct); - return struct; } @Override - public Map addToMap(SinkRecord record, Map map) { + public void addToMap(SinkRecord record, Map map) { map.put(topicFieldName, record.topic()); map.put(partitionFieldName, record.kafkaPartition()); map.put(offsetFieldName, record.kafkaOffset()); @@ -231,7 +219,6 @@ public Map addToMap(SinkRecord record, Map map) map.put(timestampFieldName, record.timestamp()); } externalFieldAppender.addToMap(record, map); - return map; } }; } From 70d8b87fa2b3fd60e7ececddd7eac5b040ac7926 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 16:04:05 -0600 Subject: [PATCH 15/17] readme typos --- kafka-connect-transforms/README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index 43ff793e..e4e5f9dc 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -27,7 +27,7 @@ The `CopyValue` SMT copies a value from one field to a new field. # KafkaMetadataTransform _(Experimental)_ -The `KafkaMetadata` injects `topic`, `partition`, `offset`, `record timestamp`. +The `KafkaMetadata` injects `topic`, `partition`, `offset`, `timestamp` which are properties are the Kafka message. ## Configuration @@ -39,10 +39,10 @@ The `KafkaMetadata` injects `topic`, `partition`, `offset`, `record timestamp`. If `nested` is on: -`_kafka_metadata.topic`, `_kafka_metadata.partition`, `kafka_metadata.offset`, `kafka_metadata.record_timestamp` +`_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_record_timestamp` +`_kafka_metdata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, `_kafka_metadata_timestamp` # DmsTransform _(Experimental)_ From 50086118c5555f7a9e6d76168bfb39211af9b2df Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 16:19:33 -0600 Subject: [PATCH 16/17] . --- .../iceberg/connect/transforms/KafkaMetadataTransform.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 index 743aa000..9a827025 100644 --- 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 @@ -195,8 +195,9 @@ public void addToSchema(SchemaBuilder builder) { builder .field(topicFieldName, Schema.STRING_SCHEMA) .field(partitionFieldName, Schema.INT32_SCHEMA) - .field(offsetFieldName, Schema.OPTIONAL_INT64_SCHEMA) + .field(offsetFieldName, Schema.INT64_SCHEMA) .field(timestampFieldName, Schema.OPTIONAL_INT64_SCHEMA); + externalFieldAppender.addToSchema(builder); } @Override From 274b38d7a48da7b234d300cc75e94364bd7897d7 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 12 Apr 2024 16:21:34 -0600 Subject: [PATCH 17/17] fix readme --- kafka-connect-transforms/README.md | 51 +++++++++++++++--------------- 1 file changed, 25 insertions(+), 26 deletions(-) diff --git a/kafka-connect-transforms/README.md b/kafka-connect-transforms/README.md index e4e5f9dc..d099bee5 100644 --- a/kafka-connect-transforms/README.md +++ b/kafka-connect-transforms/README.md @@ -24,26 +24,6 @@ The `CopyValue` SMT copies a value from one field to a new field. "transforms.copyId.target.field": "id_copy", ``` -# 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` - # DmsTransform _(Experimental)_ @@ -68,7 +48,6 @@ 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)_ @@ -85,14 +64,14 @@ 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 -| Property | Description (default value) | -|-------------|------------------------------------------| -| json.root | (false) Boolean value to start at root | +| Property | Description (default value) | +|----------------------|------------------------------------------| +| 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`. @@ -141,4 +120,24 @@ SinkRecord.value (Struct): "key" 1, "array" ["1", "two", "3"] "nested_object" Map ("some_key" : "["one", "two"]") -``` \ No newline at end of file +``` + +# 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`