From a73a96cedf465501a28db1555e21320e6b56a3e4 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 16 Mar 2023 10:33:28 -0700 Subject: [PATCH 01/50] initial commit message for added debezium date and micro timestamp coercion support --- .../iceberg/sink/IcebergChangeEvent.java | 78 +++++++++++++++---- .../sink/IcebergSinkConfiguration.java | 24 +++++- .../connect/iceberg/sink/IcebergSinkTask.java | 2 + 3 files changed, 90 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 1cb85de..005dce4 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -23,6 +23,7 @@ import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.time.LocalDate; import java.util.*; /** @@ -35,6 +36,16 @@ public class IcebergChangeEvent { private final JsonNode value; private final JsonNode key; private final JsonSchema jsonSchema; + private static boolean coerceDebeziumDate = false; + private static boolean coerceDebeziumMicroTimestamp = false; + + public static void setCoerceDebeziumDate(boolean value) { + coerceDebeziumDate = value; + } + + public static void setCoerceDebeziumMicroTimestamp(boolean value) { + coerceDebeziumMicroTimestamp = value; + } public IcebergChangeEvent(String destination, JsonNode value, @@ -97,14 +108,25 @@ private GenericRecord asIcebergRecord(Types.StructType tableFields, JsonNode dat return record; } - private Type.PrimitiveType icebergFieldType(String fieldType) { + private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeName) { switch (fieldType) { case "int8": case "int16": case "int32": // int 4 bytes - return Types.IntegerType.get(); + if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + return Types.StringType.get(); + } + else { + return Types.IntegerType.get(); + } case "int64": // long 8 bytes - return Types.LongType.get(); + if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { + return Types.StringType.get(); + } + else { + return Types.LongType.get(); + } case "float8": case "float16": case "float32": // float is represented in 32 bits, @@ -128,7 +150,10 @@ private Type.PrimitiveType icebergFieldType(String fieldType) { private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) { - LOGGER.debug("Processing Field:{} Type:{}", field.name(), field.type()); + String fieldTypeName = field.doc(); + LOGGER.debug("Processing Field:{} Type:{} Doc:{}", + field.name(), field.type(), fieldTypeName); + final Object val; switch (field.type().typeId()) { case INTEGER: // int 4 bytes @@ -147,8 +172,20 @@ private Object jsonValToIcebergVal(Types.NestedField field, val = node.isNull() ? null : node.asBoolean(); break; case STRING: - // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); + // string destination coercions based upon schema 'name' annotations + if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); + } + else { + if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { + val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); + } + else { + // if the node is not a value node (method isValueNode returns false), convert it to string. + val = node.isValueNode() ? node.asText(null) : node.toString(); + } + } break; case BINARY: try { @@ -280,7 +317,14 @@ private List icebergSchema(JsonNode eventSchema, String schem columnId++; String fieldName = jsonSchemaFieldNode.get("field").textValue(); String fieldType = jsonSchemaFieldNode.get("type").textValue(); - LOGGER.debug("Processing Field: [{}] {}.{}::{}", columnId, schemaName, fieldName, fieldType); + String fieldTypeName = ""; + JsonNode fieldTypeNameNode = jsonSchemaFieldNode.get("name"); + if (fieldTypeNameNode != null && !fieldTypeNameNode.isMissingNode()) { + fieldTypeName = fieldTypeNameNode.textValue(); + } + + LOGGER.debug("Processing Field: [{}] {}.{}::{} ({})", + columnId, schemaName, fieldName, fieldType, fieldTypeName); switch (fieldType) { case "array": JsonNode items = jsonSchemaFieldNode.get("items"); @@ -291,10 +335,10 @@ private List icebergSchema(JsonNode eventSchema, String schem throw new RuntimeException("Complex nested array types are not supported," + " array[" + listItemType + "], field " + fieldName); } - - Type.PrimitiveType item = icebergFieldType(listItemType); + // primitive coercions are not supported for list types, pass '""' for fieldTypeName + Type.PrimitiveType item = icebergFieldType(listItemType, ""); schemaColumns.add(Types.NestedField.optional( - columnId, fieldName, Types.ListType.ofOptional(++columnId, item))); + columnId, fieldName, Types.ListType.ofOptional(++columnId, item), "")); } else { throw new RuntimeException("Unexpected Array type for field " + fieldName); } @@ -304,19 +348,27 @@ private List icebergSchema(JsonNode eventSchema, String schem //break; case "struct": // create it as struct, nested type + // passing "" for NestedField `doc` attribute, + // as `doc` annotated coercions are not supported for members of struct types List subSchema = icebergSchema(jsonSchemaFieldNode, fieldName, columnId); - schemaColumns.add(Types.NestedField.optional(columnId, fieldName, Types.StructType.of(subSchema))); + schemaColumns.add(Types.NestedField.optional(columnId, fieldName, + Types.StructType.of(subSchema), "")); columnId += subSchema.size(); break; default: //primitive types - schemaColumns.add(Types.NestedField.optional(columnId, fieldName, icebergFieldType(fieldType))); + // passing fieldTypeName for NestedField `doc` attribute, + // annotation based value coercions can be made utilizing the NestedField `doc` initializer/method + schemaColumns.add(Types.NestedField.optional(columnId, fieldName, + icebergFieldType(fieldType, fieldTypeName), + fieldTypeName)); break; } } if (addSourceTsField) { columnId++; - schemaColumns.add(Types.NestedField.optional(columnId, "__source_ts", Types.TimestampType.withZone())); + schemaColumns.add(Types.NestedField.optional(columnId, "__source_ts", + Types.TimestampType.withZone(), "")); } return schemaColumns; } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index 87b74f2..f3b83bb 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -11,6 +11,8 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; +import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; + public class IcebergSinkConfiguration { public static final String UPSERT = "upsert"; public static final String UPSERT_KEEP_DELETES = "upsert.keep-deletes"; @@ -21,6 +23,8 @@ public class IcebergSinkConfiguration { public static final String TABLE_PREFIX = "table.prefix"; public static final String TABLE_AUTO_CREATE = "table.auto-create"; public static final String TABLE_SNAKE_CASE = "table.snake-case"; + public static final String COERCE_DEBEZIUM_DATE = "coerce.debezium-date"; + public static final String COERCE_DEBEZIUM_MICRO_TIMESTAMP = "coerce.debezium-micro-timestamp"; public static final String ICEBERG_PREFIX = "iceberg."; public static final String ICEBERG_TABLE_PREFIX = "iceberg.table-default"; public static final String CATALOG_NAME = ICEBERG_PREFIX + "name"; @@ -48,6 +52,11 @@ public class IcebergSinkConfiguration { "Prefix added to all table names") .define(TABLE_SNAKE_CASE, BOOLEAN, false, MEDIUM, "Coerce table names to snake_case") + .define(COERCE_DEBEZIUM_DATE, BOOLEAN, false, MEDIUM, + "Coerce int32 values with 'io.debezium.time.Date' annotation to local-date strings") + .define(COERCE_DEBEZIUM_MICRO_TIMESTAMP, BOOLEAN, false, MEDIUM, + "Coerce int64 values with 'io.debezium.time.MicroTimestamp' annotation to" + + "iso datetime strings") .define(CATALOG_NAME, STRING, "default", MEDIUM, "Iceberg catalog name") .define(CATALOG_IMPL, STRING, null, MEDIUM, @@ -102,10 +111,18 @@ public boolean isTableSnakeCase() { return parsedConfig.getBoolean(TABLE_SNAKE_CASE); } + public boolean isCoerceDebeziumDate() { + return parsedConfig.getBoolean(COERCE_DEBEZIUM_DATE); + } + + public boolean isCoerceDebeziumMicroTimestamp() { + return parsedConfig.getBoolean(COERCE_DEBEZIUM_MICRO_TIMESTAMP); + } + public String getCatalogName() { return parsedConfig.getString(CATALOG_NAME); } - + public Map getIcebergCatalogConfiguration() { return getConfiguration(ICEBERG_PREFIX); } @@ -129,4 +146,9 @@ public static ConfigDef getConfigDef() { public Map getProperties() { return properties; } + + public void configureChangeEvent() { + IcebergChangeEvent.setCoerceDebeziumDate(this.isCoerceDebeziumDate()); + IcebergChangeEvent.setCoerceDebeziumMicroTimestamp(this.isCoerceDebeziumMicroTimestamp()); + } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java index 894fcf3..3b2f8c1 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java @@ -26,6 +26,8 @@ public String version() { public void start(Map properties) { LOGGER.info("Task starting"); IcebergSinkConfiguration configuration = new IcebergSinkConfiguration(properties); + // provide type coercion configuration IcebergChangeEvent + configuration.configureChangeEvent(); Catalog icebergCatalog = IcebergCatalogFactory.create(configuration); IcebergTableOperator icebergTableOperator = IcebergTableOperatorFactory.create(configuration); SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(); From c2626ad447738136dd518380e6d9814cf5c7eff2 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:21:00 -0700 Subject: [PATCH 02/50] repair unit tests after adding type annotation information to schema --- .../kafka/connect/iceberg/sink/TestIcebergUtil.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 2570de0..5287a46 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -47,7 +47,7 @@ public void testNestedJsonRecord() throws JsonProcessingException { MAPPER.readTree(serdeWithSchema).get("payload"), null, MAPPER.readTree(serdeWithSchema).get("schema"), null); Schema schema = e.icebergSchema(); - assertTrue(schema.toString().contains("before: optional struct<2: id: optional int, 3: first_name: optional string, " + + assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + "4:")); } @@ -70,8 +70,7 @@ public void testNestedArrayJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, MAPPER.readTree(unwrapWithArraySchema).get("schema"), null); Schema schema = e.icebergSchema(); - assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string, 2: pay_by_quarter: optional list, 4: schedule: optional list, 6:")); - System.out.println(schema.asStruct()); + assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); @@ -101,7 +100,7 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); - assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string, 4: srid: optional int>")); + assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); GenericRecord g = (GenericRecord) record.getField("g"); GenericRecord h = (GenericRecord) record.getField("h"); assertEquals("AQEAAAAAAAAAAADwPwAAAAAAAPA/", g.get(0, Types.StringType.get().typeId().javaClass())); From f3268fc8e74d0384eb5d1cd8a30d44ebb4fe735a Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:22:30 -0700 Subject: [PATCH 03/50] Added unit tests for Debezium time coercions. Unit tests demonstrate that coercions should not applied by default, without configuration changes, should not be applied when explicitly disabled via configuration, and are only applied when explicitly configured via IcebergSinkConfiguration. Both values and types are tested. --- .../connect/iceberg/sink/TestIcebergUtil.java | 68 +++++++++++++++++++ .../sink/testresources/TestConfig.java | 5 ++ .../json/debezium-annotated-schema.json | 37 ++++++++++ 3 files changed, 110 insertions(+) create mode 100644 src/test/resources/json/debezium-annotated-schema.json diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 5287a46..f5dfb4b 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -40,6 +40,7 @@ class TestIcebergUtil { final String unwrapWithGeomSchema = Testing.Files.readResourceAsString("json/serde-with-schema_geom.json"); final String unwrapWithArraySchema = Testing.Files.readResourceAsString("json/serde-with-array.json"); final String unwrapWithArraySchema2 = Testing.Files.readResourceAsString("json/serde-with-array2.json"); + final String debeziumTimeCoercionSchema = Testing.Files.readResourceAsString("json/debezium-annotated-schema.json"); @Test public void testNestedJsonRecord() throws JsonProcessingException { @@ -130,6 +131,73 @@ public void valuePayloadWithSchemaAsJsonNode() { assertFalse(deserializedSchema.has("schema")); } + @Test + public void coerceDebeziumTimeTypesDefaultBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesDisabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "false") + .withCustomProperty("coerce.debezium-micro-timestamp", "false") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "true") + .withCustomProperty("coerce.debezium-micro-timestamp", "true") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional string (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional string (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("2182-08-20")); + assertTrue(recordString.contains("2182-08-19T21:50:56.016196Z")); + } + @Test public void createIcebergTablesWithCustomProperties(@TempDir Path localWarehouseDir) { IcebergSinkConfiguration config = TestConfig.builder() diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java index aa619ab..4f0883e 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java @@ -64,6 +64,11 @@ public Builder withCustomCatalogProperty(String key, String value) { return this; } + public Builder withCustomProperty(String key, String value) { + properties.put(key, value); + return this; + } + public IcebergSinkConfiguration build() { return new IcebergSinkConfiguration(properties); } diff --git a/src/test/resources/json/debezium-annotated-schema.json b/src/test/resources/json/debezium-annotated-schema.json new file mode 100644 index 0000000..1e3032d --- /dev/null +++ b/src/test/resources/json/debezium-annotated-schema.json @@ -0,0 +1,37 @@ +{ + "schema": { + "type": "struct", + "fields": [ + { + "type": "int32", + "optional": false, + "field": "id" + }, + { + "type": "int32", + "optional": false, + "name": "io.debezium.time.Date", + "version": 1, + "field": "ship_date" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "field": "ship_timestamp" + } + ], + "optional": false, + "name": "testc.ship.time.Value" + }, + "payload": { + "id": 10003, + "ship_date": 77663, + "ship_timestamp": 6710075456016196, + "__op": "r", + "__table": "time", + "__lsn": 33832960, + "__source_ts_ms": 1596309876678, + "__deleted": "false" + } +} From 71033f2f9c1ccd07df1da1dfd98220ee722567e6 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:34:41 -0700 Subject: [PATCH 04/50] updated CHANGELOG.md --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8d83477..b937da1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,11 +2,12 @@ ## [Unreleased] +- Added support for coercion of Debezium time types to formatted `string` values using type annotations. + ## [0.2.4] - 2023-03-13 - Added support for `double` primitive type fields. - Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. - ## [0.2.2] - 2023-02-17 - Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties From d812d448085c5c41f6a4c57cfe79ce7c1ed3224d Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Tue, 21 Mar 2023 09:51:37 -0700 Subject: [PATCH 05/50] formatting adjustments --- .../connect/iceberg/sink/IcebergChangeEvent.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 005dce4..875101c 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -152,7 +152,7 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) { String fieldTypeName = field.doc(); LOGGER.debug("Processing Field:{} Type:{} Doc:{}", - field.name(), field.type(), fieldTypeName); + field.name(), field.type(), fieldTypeName); final Object val; switch (field.type().typeId()) { @@ -173,17 +173,21 @@ private Object jsonValToIcebergVal(Types.NestedField field, break; case STRING: // string destination coercions based upon schema 'name' annotations - if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { - val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); + if (IcebergChangeEvent.coerceDebeziumDate && + fieldTypeName.equals("io.debezium.time.Date")) { + val = node.isNull() ? null + : LocalDate.ofEpochDay(node.asInt()).toString(); } else { if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { - val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); + val = node.isNull() ? null + : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); } else { // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); + val = node.isValueNode() ? node.asText(null) + : node.toString(); } } break; From 35237c18002cb59dab7980da74b91d5fa44850b8 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 16 Mar 2023 10:33:28 -0700 Subject: [PATCH 06/50] initial commit message for added debezium date and micro timestamp coercion support --- .../iceberg/sink/IcebergChangeEvent.java | 78 +++++++++++++++---- .../sink/IcebergSinkConfiguration.java | 24 +++++- .../connect/iceberg/sink/IcebergSinkTask.java | 2 + 3 files changed, 90 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 1cb85de..005dce4 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -23,6 +23,7 @@ import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.time.LocalDate; import java.util.*; /** @@ -35,6 +36,16 @@ public class IcebergChangeEvent { private final JsonNode value; private final JsonNode key; private final JsonSchema jsonSchema; + private static boolean coerceDebeziumDate = false; + private static boolean coerceDebeziumMicroTimestamp = false; + + public static void setCoerceDebeziumDate(boolean value) { + coerceDebeziumDate = value; + } + + public static void setCoerceDebeziumMicroTimestamp(boolean value) { + coerceDebeziumMicroTimestamp = value; + } public IcebergChangeEvent(String destination, JsonNode value, @@ -97,14 +108,25 @@ private GenericRecord asIcebergRecord(Types.StructType tableFields, JsonNode dat return record; } - private Type.PrimitiveType icebergFieldType(String fieldType) { + private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeName) { switch (fieldType) { case "int8": case "int16": case "int32": // int 4 bytes - return Types.IntegerType.get(); + if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + return Types.StringType.get(); + } + else { + return Types.IntegerType.get(); + } case "int64": // long 8 bytes - return Types.LongType.get(); + if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { + return Types.StringType.get(); + } + else { + return Types.LongType.get(); + } case "float8": case "float16": case "float32": // float is represented in 32 bits, @@ -128,7 +150,10 @@ private Type.PrimitiveType icebergFieldType(String fieldType) { private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) { - LOGGER.debug("Processing Field:{} Type:{}", field.name(), field.type()); + String fieldTypeName = field.doc(); + LOGGER.debug("Processing Field:{} Type:{} Doc:{}", + field.name(), field.type(), fieldTypeName); + final Object val; switch (field.type().typeId()) { case INTEGER: // int 4 bytes @@ -147,8 +172,20 @@ private Object jsonValToIcebergVal(Types.NestedField field, val = node.isNull() ? null : node.asBoolean(); break; case STRING: - // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); + // string destination coercions based upon schema 'name' annotations + if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); + } + else { + if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { + val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); + } + else { + // if the node is not a value node (method isValueNode returns false), convert it to string. + val = node.isValueNode() ? node.asText(null) : node.toString(); + } + } break; case BINARY: try { @@ -280,7 +317,14 @@ private List icebergSchema(JsonNode eventSchema, String schem columnId++; String fieldName = jsonSchemaFieldNode.get("field").textValue(); String fieldType = jsonSchemaFieldNode.get("type").textValue(); - LOGGER.debug("Processing Field: [{}] {}.{}::{}", columnId, schemaName, fieldName, fieldType); + String fieldTypeName = ""; + JsonNode fieldTypeNameNode = jsonSchemaFieldNode.get("name"); + if (fieldTypeNameNode != null && !fieldTypeNameNode.isMissingNode()) { + fieldTypeName = fieldTypeNameNode.textValue(); + } + + LOGGER.debug("Processing Field: [{}] {}.{}::{} ({})", + columnId, schemaName, fieldName, fieldType, fieldTypeName); switch (fieldType) { case "array": JsonNode items = jsonSchemaFieldNode.get("items"); @@ -291,10 +335,10 @@ private List icebergSchema(JsonNode eventSchema, String schem throw new RuntimeException("Complex nested array types are not supported," + " array[" + listItemType + "], field " + fieldName); } - - Type.PrimitiveType item = icebergFieldType(listItemType); + // primitive coercions are not supported for list types, pass '""' for fieldTypeName + Type.PrimitiveType item = icebergFieldType(listItemType, ""); schemaColumns.add(Types.NestedField.optional( - columnId, fieldName, Types.ListType.ofOptional(++columnId, item))); + columnId, fieldName, Types.ListType.ofOptional(++columnId, item), "")); } else { throw new RuntimeException("Unexpected Array type for field " + fieldName); } @@ -304,19 +348,27 @@ private List icebergSchema(JsonNode eventSchema, String schem //break; case "struct": // create it as struct, nested type + // passing "" for NestedField `doc` attribute, + // as `doc` annotated coercions are not supported for members of struct types List subSchema = icebergSchema(jsonSchemaFieldNode, fieldName, columnId); - schemaColumns.add(Types.NestedField.optional(columnId, fieldName, Types.StructType.of(subSchema))); + schemaColumns.add(Types.NestedField.optional(columnId, fieldName, + Types.StructType.of(subSchema), "")); columnId += subSchema.size(); break; default: //primitive types - schemaColumns.add(Types.NestedField.optional(columnId, fieldName, icebergFieldType(fieldType))); + // passing fieldTypeName for NestedField `doc` attribute, + // annotation based value coercions can be made utilizing the NestedField `doc` initializer/method + schemaColumns.add(Types.NestedField.optional(columnId, fieldName, + icebergFieldType(fieldType, fieldTypeName), + fieldTypeName)); break; } } if (addSourceTsField) { columnId++; - schemaColumns.add(Types.NestedField.optional(columnId, "__source_ts", Types.TimestampType.withZone())); + schemaColumns.add(Types.NestedField.optional(columnId, "__source_ts", + Types.TimestampType.withZone(), "")); } return schemaColumns; } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index 87b74f2..f3b83bb 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -11,6 +11,8 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; +import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; + public class IcebergSinkConfiguration { public static final String UPSERT = "upsert"; public static final String UPSERT_KEEP_DELETES = "upsert.keep-deletes"; @@ -21,6 +23,8 @@ public class IcebergSinkConfiguration { public static final String TABLE_PREFIX = "table.prefix"; public static final String TABLE_AUTO_CREATE = "table.auto-create"; public static final String TABLE_SNAKE_CASE = "table.snake-case"; + public static final String COERCE_DEBEZIUM_DATE = "coerce.debezium-date"; + public static final String COERCE_DEBEZIUM_MICRO_TIMESTAMP = "coerce.debezium-micro-timestamp"; public static final String ICEBERG_PREFIX = "iceberg."; public static final String ICEBERG_TABLE_PREFIX = "iceberg.table-default"; public static final String CATALOG_NAME = ICEBERG_PREFIX + "name"; @@ -48,6 +52,11 @@ public class IcebergSinkConfiguration { "Prefix added to all table names") .define(TABLE_SNAKE_CASE, BOOLEAN, false, MEDIUM, "Coerce table names to snake_case") + .define(COERCE_DEBEZIUM_DATE, BOOLEAN, false, MEDIUM, + "Coerce int32 values with 'io.debezium.time.Date' annotation to local-date strings") + .define(COERCE_DEBEZIUM_MICRO_TIMESTAMP, BOOLEAN, false, MEDIUM, + "Coerce int64 values with 'io.debezium.time.MicroTimestamp' annotation to" + + "iso datetime strings") .define(CATALOG_NAME, STRING, "default", MEDIUM, "Iceberg catalog name") .define(CATALOG_IMPL, STRING, null, MEDIUM, @@ -102,10 +111,18 @@ public boolean isTableSnakeCase() { return parsedConfig.getBoolean(TABLE_SNAKE_CASE); } + public boolean isCoerceDebeziumDate() { + return parsedConfig.getBoolean(COERCE_DEBEZIUM_DATE); + } + + public boolean isCoerceDebeziumMicroTimestamp() { + return parsedConfig.getBoolean(COERCE_DEBEZIUM_MICRO_TIMESTAMP); + } + public String getCatalogName() { return parsedConfig.getString(CATALOG_NAME); } - + public Map getIcebergCatalogConfiguration() { return getConfiguration(ICEBERG_PREFIX); } @@ -129,4 +146,9 @@ public static ConfigDef getConfigDef() { public Map getProperties() { return properties; } + + public void configureChangeEvent() { + IcebergChangeEvent.setCoerceDebeziumDate(this.isCoerceDebeziumDate()); + IcebergChangeEvent.setCoerceDebeziumMicroTimestamp(this.isCoerceDebeziumMicroTimestamp()); + } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java index 894fcf3..3b2f8c1 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java @@ -26,6 +26,8 @@ public String version() { public void start(Map properties) { LOGGER.info("Task starting"); IcebergSinkConfiguration configuration = new IcebergSinkConfiguration(properties); + // provide type coercion configuration IcebergChangeEvent + configuration.configureChangeEvent(); Catalog icebergCatalog = IcebergCatalogFactory.create(configuration); IcebergTableOperator icebergTableOperator = IcebergTableOperatorFactory.create(configuration); SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(); From 8820b1af6132e22640a440563ead41c93fac4098 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:21:00 -0700 Subject: [PATCH 07/50] repair unit tests after adding type annotation information to schema --- .../kafka/connect/iceberg/sink/TestIcebergUtil.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 2570de0..5287a46 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -47,7 +47,7 @@ public void testNestedJsonRecord() throws JsonProcessingException { MAPPER.readTree(serdeWithSchema).get("payload"), null, MAPPER.readTree(serdeWithSchema).get("schema"), null); Schema schema = e.icebergSchema(); - assertTrue(schema.toString().contains("before: optional struct<2: id: optional int, 3: first_name: optional string, " + + assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + "4:")); } @@ -70,8 +70,7 @@ public void testNestedArrayJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, MAPPER.readTree(unwrapWithArraySchema).get("schema"), null); Schema schema = e.icebergSchema(); - assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string, 2: pay_by_quarter: optional list, 4: schedule: optional list, 6:")); - System.out.println(schema.asStruct()); + assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); @@ -101,7 +100,7 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); - assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string, 4: srid: optional int>")); + assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); GenericRecord g = (GenericRecord) record.getField("g"); GenericRecord h = (GenericRecord) record.getField("h"); assertEquals("AQEAAAAAAAAAAADwPwAAAAAAAPA/", g.get(0, Types.StringType.get().typeId().javaClass())); From 3f24c77a850466b1fce1008e7956c7ce34e4fa59 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:22:30 -0700 Subject: [PATCH 08/50] Added unit tests for Debezium time coercions. Unit tests demonstrate that coercions should not applied by default, without configuration changes, should not be applied when explicitly disabled via configuration, and are only applied when explicitly configured via IcebergSinkConfiguration. Both values and types are tested. --- .../connect/iceberg/sink/TestIcebergUtil.java | 68 +++++++++++++++++++ .../sink/testresources/TestConfig.java | 5 ++ .../json/debezium-annotated-schema.json | 37 ++++++++++ 3 files changed, 110 insertions(+) create mode 100644 src/test/resources/json/debezium-annotated-schema.json diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 5287a46..f5dfb4b 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -40,6 +40,7 @@ class TestIcebergUtil { final String unwrapWithGeomSchema = Testing.Files.readResourceAsString("json/serde-with-schema_geom.json"); final String unwrapWithArraySchema = Testing.Files.readResourceAsString("json/serde-with-array.json"); final String unwrapWithArraySchema2 = Testing.Files.readResourceAsString("json/serde-with-array2.json"); + final String debeziumTimeCoercionSchema = Testing.Files.readResourceAsString("json/debezium-annotated-schema.json"); @Test public void testNestedJsonRecord() throws JsonProcessingException { @@ -130,6 +131,73 @@ public void valuePayloadWithSchemaAsJsonNode() { assertFalse(deserializedSchema.has("schema")); } + @Test + public void coerceDebeziumTimeTypesDefaultBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesDisabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "false") + .withCustomProperty("coerce.debezium-micro-timestamp", "false") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "true") + .withCustomProperty("coerce.debezium-micro-timestamp", "true") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional string (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional string (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("2182-08-20")); + assertTrue(recordString.contains("2182-08-19T21:50:56.016196Z")); + } + @Test public void createIcebergTablesWithCustomProperties(@TempDir Path localWarehouseDir) { IcebergSinkConfiguration config = TestConfig.builder() diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java index aa619ab..4f0883e 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java @@ -64,6 +64,11 @@ public Builder withCustomCatalogProperty(String key, String value) { return this; } + public Builder withCustomProperty(String key, String value) { + properties.put(key, value); + return this; + } + public IcebergSinkConfiguration build() { return new IcebergSinkConfiguration(properties); } diff --git a/src/test/resources/json/debezium-annotated-schema.json b/src/test/resources/json/debezium-annotated-schema.json new file mode 100644 index 0000000..1e3032d --- /dev/null +++ b/src/test/resources/json/debezium-annotated-schema.json @@ -0,0 +1,37 @@ +{ + "schema": { + "type": "struct", + "fields": [ + { + "type": "int32", + "optional": false, + "field": "id" + }, + { + "type": "int32", + "optional": false, + "name": "io.debezium.time.Date", + "version": 1, + "field": "ship_date" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTimestamp", + "field": "ship_timestamp" + } + ], + "optional": false, + "name": "testc.ship.time.Value" + }, + "payload": { + "id": 10003, + "ship_date": 77663, + "ship_timestamp": 6710075456016196, + "__op": "r", + "__table": "time", + "__lsn": 33832960, + "__source_ts_ms": 1596309876678, + "__deleted": "false" + } +} From c64ba157c0de0a055399bd4f6ffe19b8780a83e6 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:34:41 -0700 Subject: [PATCH 09/50] updated CHANGELOG.md --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f398d13..63964fb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,8 @@ ## [Unreleased] +- Added support for coercion of Debezium time types to formatted `string` values using type annotations. + ## [0.2.5] - 2023-03-20 - Reverted pom.xml groupid @@ -10,7 +12,6 @@ - Added support for `double` primitive type fields. - Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. - ## [0.2.2] - 2023-02-17 - Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties From beb6174d2089882c8c4a06831c46ff9098e5d145 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Tue, 21 Mar 2023 09:51:37 -0700 Subject: [PATCH 10/50] formatting adjustments --- .../connect/iceberg/sink/IcebergChangeEvent.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 005dce4..875101c 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -152,7 +152,7 @@ private Object jsonValToIcebergVal(Types.NestedField field, JsonNode node) { String fieldTypeName = field.doc(); LOGGER.debug("Processing Field:{} Type:{} Doc:{}", - field.name(), field.type(), fieldTypeName); + field.name(), field.type(), fieldTypeName); final Object val; switch (field.type().typeId()) { @@ -173,17 +173,21 @@ private Object jsonValToIcebergVal(Types.NestedField field, break; case STRING: // string destination coercions based upon schema 'name' annotations - if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { - val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); + if (IcebergChangeEvent.coerceDebeziumDate && + fieldTypeName.equals("io.debezium.time.Date")) { + val = node.isNull() ? null + : LocalDate.ofEpochDay(node.asInt()).toString(); } else { if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { - val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); + val = node.isNull() ? null + : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); } else { // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); + val = node.isValueNode() ? node.asText(null) + : node.toString(); } } break; From a258fb8da269aab5f54d25c13bc2612adc98d792 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 7 Apr 2023 10:01:31 -0700 Subject: [PATCH 11/50] use first class date types --- .../iceberg/sink/IcebergChangeEvent.java | 33 ++++++++----------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 875101c..5e07d5a 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -24,6 +24,7 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.LocalDate; +import java.time.LocalDateTime; import java.util.*; /** @@ -114,7 +115,7 @@ private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeNa case "int16": case "int32": // int 4 bytes if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { - return Types.StringType.get(); + return Types.DateType.get(); } else { return Types.IntegerType.get(); @@ -122,7 +123,7 @@ private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeNa case "int64": // long 8 bytes if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { - return Types.StringType.get(); + return Types.TimestampType.withoutZone(); } else { return Types.LongType.get(); @@ -171,25 +172,17 @@ private Object jsonValToIcebergVal(Types.NestedField field, case BOOLEAN: val = node.isNull() ? null : node.asBoolean(); break; + case DATE: + val = node.isNull() ? null + : LocalDate.ofEpochDay(node.asInt()); + break; + case TIMESTAMP: + val = node.isNull() ? null + : LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, node.asLong() * 1000), ZoneOffset.UTC); + break; case STRING: - // string destination coercions based upon schema 'name' annotations - if (IcebergChangeEvent.coerceDebeziumDate && - fieldTypeName.equals("io.debezium.time.Date")) { - val = node.isNull() ? null - : LocalDate.ofEpochDay(node.asInt()).toString(); - } - else { - if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && - fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { - val = node.isNull() ? null - : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); - } - else { - // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) - : node.toString(); - } - } + // if the node is not a value node (method isValueNode returns false), convert it to string. + val = node.isValueNode() ? node.asText(null) : node.toString(); break; case BINARY: try { From ab6c2c6e9cb2b969e77d06fe042b48208f3b4dfc Mon Sep 17 00:00:00 2001 From: dacreify Date: Fri, 7 Apr 2023 10:43:46 -0700 Subject: [PATCH 12/50] adapt tests to strong typing --- .../connect/iceberg/sink/TestIcebergUtil.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index f5dfb4b..2e48b17 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -27,6 +27,8 @@ import java.io.IOException; import java.nio.file.Path; +import java.time.LocalDate; +import java.time.LocalDateTime; import java.util.Collections; import java.util.List; import java.util.Set; @@ -188,14 +190,18 @@ public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseD MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); - String schemaString = schema.toString(); - String recordString = record.toString(); - assertTrue(schemaString.contains("ship_date: optional string (io.debezium.time.Date)")); - assertTrue(schemaString.contains("ship_timestamp: optional string (io.debezium.time.MicroTimestamp)")); - assertTrue(recordString.contains("2182-08-20")); - assertTrue(recordString.contains("2182-08-19T21:50:56.016196Z")); + Types.NestedField ship_date = schema.findField("ship_date"); + assertEquals(ship_date.type(), Types.DateType.get()); + assertEquals(ship_date.doc(), "io.debezium.time.Date"); + + Types.NestedField ship_timestamp = schema.findField("ship_timestamp"); + assertEquals(ship_timestamp.type(), Types.TimestampType.withoutZone()); + assertEquals(ship_timestamp.doc(), "io.debezium.time.MicroTimestamp"); + + GenericRecord record = e.asIcebergRecord(schema); + assertEquals(record.getField("ship_date"), LocalDate.parse("2182-08-20")); + assertEquals(record.getField("ship_timestamp"), LocalDateTime.parse("2182-08-19T21:50:56.016196")); } @Test From 67689cb874d35f982b2905c586b04398074f8e96 Mon Sep 17 00:00:00 2001 From: dacreify Date: Wed, 12 Apr 2023 13:29:53 -0700 Subject: [PATCH 13/50] add rich temporal support for zoned timestamps and time values, plumb through config object to events --- .../iceberg/sink/IcebergChangeEvent.java | 72 +++++++--- .../sink/IcebergSinkConfiguration.java | 26 +--- .../connect/iceberg/sink/IcebergSinkTask.java | 4 +- ...nkRecordToIcebergChangeEventConverter.java | 7 +- ...dToIcebergChangeEventConverterFactory.java | 7 +- .../connect/iceberg/sink/TestIcebergUtil.java | 129 +++++++++++------- .../IcebergChangeEventBuilder.java | 8 +- .../json/debezium-annotated-schema.json | 25 ++++ 8 files changed, 179 insertions(+), 99 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 5e07d5a..2e038a3 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -20,11 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.LocalDate; -import java.time.LocalDateTime; +import java.time.*; import java.util.*; /** @@ -37,25 +33,17 @@ public class IcebergChangeEvent { private final JsonNode value; private final JsonNode key; private final JsonSchema jsonSchema; - private static boolean coerceDebeziumDate = false; - private static boolean coerceDebeziumMicroTimestamp = false; - - public static void setCoerceDebeziumDate(boolean value) { - coerceDebeziumDate = value; - } - - public static void setCoerceDebeziumMicroTimestamp(boolean value) { - coerceDebeziumMicroTimestamp = value; - } + private final IcebergSinkConfiguration configuration; public IcebergChangeEvent(String destination, JsonNode value, JsonNode key, JsonNode valueSchema, - JsonNode keySchema) { + JsonNode keySchema, IcebergSinkConfiguration configuration) { this.destination = destination; this.value = value; this.key = key; + this.configuration = configuration; this.jsonSchema = new JsonSchema(valueSchema, keySchema); } @@ -114,17 +102,22 @@ private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeNa case "int8": case "int16": case "int32": // int 4 bytes - if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + if (configuration.isRichTemporalTypes() && + fieldTypeName.equals("io.debezium.time.Date")) { return Types.DateType.get(); } else { return Types.IntegerType.get(); } case "int64": // long 8 bytes - if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + if (configuration.isRichTemporalTypes() && fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { return Types.TimestampType.withoutZone(); } + else if (configuration.isRichTemporalTypes() && + fieldTypeName.equals("io.debezium.time.MicroTime")) { + return Types.TimeType.get(); + } else { return Types.LongType.get(); } @@ -139,7 +132,17 @@ private Type.PrimitiveType icebergFieldType(String fieldType, String fieldTypeNa case "boolean": return Types.BooleanType.get(); case "string": - return Types.StringType.get(); + if (configuration.isRichTemporalTypes() && + fieldTypeName.equals("io.debezium.time.ZonedTimestamp")) { + return Types.TimestampType.withZone(); + } + else if (configuration.isRichTemporalTypes() && + fieldTypeName.equals("io.debezium.time.ZonedTime")) { + return Types.TimeType.get(); + } + else { + return Types.StringType.get(); + } case "bytes": return Types.BinaryType.get(); default: @@ -177,8 +180,35 @@ private Object jsonValToIcebergVal(Types.NestedField field, : LocalDate.ofEpochDay(node.asInt()); break; case TIMESTAMP: - val = node.isNull() ? null - : LocalDateTime.ofInstant(Instant.ofEpochSecond(0L, node.asLong() * 1000), ZoneOffset.UTC); + if (node.isTextual()) { + val = ZonedDateTime.parse(node.asText()); + } + else if (node.isNumber()) { + Instant instant = Instant.ofEpochSecond(0L, node.asLong() * 1000); + val = LocalDateTime.ofInstant(instant, ZoneOffset.UTC); + } + else if (node.isNull()){ + val = null; + } + else { + throw new RuntimeException("Unrecognized JSON node type for Iceberg type TIMESTAMP: " + node.getNodeType()); + } + break; + case TIME: + if (node.isLong()) { + val = LocalTime.ofNanoOfDay(node.asLong() * 1000); + } + else if (node.isTextual()) { + // Debezium converts ZonedTime values to UTC on capture, so no information is lost by converting them + // to LocalTimes here. Iceberg doesn't support a ZonedTime equivalent anyway. + val = OffsetTime.parse(node.asText()).toLocalTime(); + } + else if (node.isNull()){ + val = null; + } + else { + throw new RuntimeException("Unrecognized JSON node type for Iceberg type TIME: " + node.getNodeType()); + } break; case STRING: // if the node is not a value node (method isValueNode returns false), convert it to string. diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index f3b83bb..f871d74 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -11,8 +11,6 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; -import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; - public class IcebergSinkConfiguration { public static final String UPSERT = "upsert"; public static final String UPSERT_KEEP_DELETES = "upsert.keep-deletes"; @@ -23,8 +21,7 @@ public class IcebergSinkConfiguration { public static final String TABLE_PREFIX = "table.prefix"; public static final String TABLE_AUTO_CREATE = "table.auto-create"; public static final String TABLE_SNAKE_CASE = "table.snake-case"; - public static final String COERCE_DEBEZIUM_DATE = "coerce.debezium-date"; - public static final String COERCE_DEBEZIUM_MICRO_TIMESTAMP = "coerce.debezium-micro-timestamp"; + public static final String RICH_TEMPORAL_TYPES = "rich-temporal-types"; public static final String ICEBERG_PREFIX = "iceberg."; public static final String ICEBERG_TABLE_PREFIX = "iceberg.table-default"; public static final String CATALOG_NAME = ICEBERG_PREFIX + "name"; @@ -52,11 +49,9 @@ public class IcebergSinkConfiguration { "Prefix added to all table names") .define(TABLE_SNAKE_CASE, BOOLEAN, false, MEDIUM, "Coerce table names to snake_case") - .define(COERCE_DEBEZIUM_DATE, BOOLEAN, false, MEDIUM, - "Coerce int32 values with 'io.debezium.time.Date' annotation to local-date strings") - .define(COERCE_DEBEZIUM_MICRO_TIMESTAMP, BOOLEAN, false, MEDIUM, - "Coerce int64 values with 'io.debezium.time.MicroTimestamp' annotation to" + - "iso datetime strings") + .define(RICH_TEMPORAL_TYPES, BOOLEAN, false, MEDIUM, + "Coerce Debezium Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime values " + + "from JSON primitives to their corresponding Iceberg rich types") .define(CATALOG_NAME, STRING, "default", MEDIUM, "Iceberg catalog name") .define(CATALOG_IMPL, STRING, null, MEDIUM, @@ -111,12 +106,8 @@ public boolean isTableSnakeCase() { return parsedConfig.getBoolean(TABLE_SNAKE_CASE); } - public boolean isCoerceDebeziumDate() { - return parsedConfig.getBoolean(COERCE_DEBEZIUM_DATE); - } - - public boolean isCoerceDebeziumMicroTimestamp() { - return parsedConfig.getBoolean(COERCE_DEBEZIUM_MICRO_TIMESTAMP); + public boolean isRichTemporalTypes() { + return parsedConfig.getBoolean(RICH_TEMPORAL_TYPES); } public String getCatalogName() { @@ -146,9 +137,4 @@ public static ConfigDef getConfigDef() { public Map getProperties() { return properties; } - - public void configureChangeEvent() { - IcebergChangeEvent.setCoerceDebeziumDate(this.isCoerceDebeziumDate()); - IcebergChangeEvent.setCoerceDebeziumMicroTimestamp(this.isCoerceDebeziumMicroTimestamp()); - } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java index 3b2f8c1..a41cb4f 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java @@ -26,11 +26,9 @@ public String version() { public void start(Map properties) { LOGGER.info("Task starting"); IcebergSinkConfiguration configuration = new IcebergSinkConfiguration(properties); - // provide type coercion configuration IcebergChangeEvent - configuration.configureChangeEvent(); Catalog icebergCatalog = IcebergCatalogFactory.create(configuration); IcebergTableOperator icebergTableOperator = IcebergTableOperatorFactory.create(configuration); - SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(); + SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(configuration); consumer = new IcebergChangeConsumer(configuration, icebergCatalog, icebergTableOperator, converter); } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java index 8c24aac..e35ea64 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java @@ -3,6 +3,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; +import com.getindata.kafka.connect.iceberg.sink.IcebergSinkConfiguration; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; @@ -19,17 +20,19 @@ public class SinkRecordToIcebergChangeEventConverter { private final JsonConverter valueJsonConverter; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; + private final IcebergSinkConfiguration configuration; public SinkRecordToIcebergChangeEventConverter(Transformation extractNewRecordStateTransformation, JsonConverter keyJsonConverter, JsonConverter valueJsonConverter, Deserializer keyDeserializer, - Deserializer valueDeserializer) { + Deserializer valueDeserializer, IcebergSinkConfiguration configuration) { this.extractNewRecordStateTransformation = extractNewRecordStateTransformation; this.keyJsonConverter = keyJsonConverter; this.valueJsonConverter = valueJsonConverter; this.keyDeserializer = keyDeserializer; this.valueDeserializer = valueDeserializer; + this.configuration = configuration; } public IcebergChangeEvent convert(SinkRecord record) { @@ -41,7 +44,7 @@ public IcebergChangeEvent convert(SinkRecord record) { JsonNode value = getValue(unwrappedRecord.topic(), valueDeserializer, valueBytes); JsonNode valueSchema = getSchema(valueBytes); - return new IcebergChangeEvent(unwrappedRecord.topic(), value, key, valueSchema, keySchema); + return new IcebergChangeEvent(unwrappedRecord.topic(), value, key, valueSchema, keySchema, configuration); } private JsonNode getValue(String topic, Deserializer deserializer, byte[] bytes) { diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverterFactory.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverterFactory.java index deae7d6..16faa8b 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverterFactory.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverterFactory.java @@ -1,13 +1,14 @@ package com.getindata.kafka.connect.iceberg.sink.converter; import com.fasterxml.jackson.databind.JsonNode; +import com.getindata.kafka.connect.iceberg.sink.IcebergSinkConfiguration; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.Transformation; public class SinkRecordToIcebergChangeEventConverterFactory { - public static SinkRecordToIcebergChangeEventConverter create() { + public static SinkRecordToIcebergChangeEventConverter create(IcebergSinkConfiguration configuration) { Transformation extractNewRecordStateTransformation = ExtractNewRecordStateTransformationFactory.create(); JsonConverter keyJsonConverter = JsonConverterFactory.create(true); JsonConverter valueJsonConverter = JsonConverterFactory.create(false); @@ -17,7 +18,7 @@ public static SinkRecordToIcebergChangeEventConverter create() { extractNewRecordStateTransformation, keyJsonConverter, valueJsonConverter, - keyDeserializer, valueDeserializer - ); + keyDeserializer, valueDeserializer, + configuration); } } diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 2e48b17..2e0a1dc 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -27,9 +27,9 @@ import java.io.IOException; import java.nio.file.Path; -import java.time.LocalDate; -import java.time.LocalDateTime; +import java.time.*; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Set; @@ -44,11 +44,13 @@ class TestIcebergUtil { final String unwrapWithArraySchema2 = Testing.Files.readResourceAsString("json/serde-with-array2.json"); final String debeziumTimeCoercionSchema = Testing.Files.readResourceAsString("json/debezium-annotated-schema.json"); + private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); + @Test public void testNestedJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(serdeWithSchema).get("payload"), null, - MAPPER.readTree(serdeWithSchema).get("schema"), null); + MAPPER.readTree(serdeWithSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + "4:")); @@ -58,20 +60,18 @@ public void testNestedJsonRecord() throws JsonProcessingException { public void testUnwrapJsonRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithSchema).get("payload"), null, - MAPPER.readTree(unwrapWithSchema).get("schema"), null); + MAPPER.readTree(unwrapWithSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); assertEquals("orders", record.getField("__table").toString()); assertEquals(16850, record.getField("order_date")); - System.out.println(schema); - System.out.println(record); } @Test public void testNestedArrayJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema).get("schema"), null); + MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); @@ -87,7 +87,7 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { assertThrows(RuntimeException.class, () -> { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema2).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null); + MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); System.out.println(schema.asStruct()); System.out.println(schema); @@ -100,7 +100,7 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { public void testNestedGeomJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithGeomSchema).get("payload"), null, - MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null); + MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); @@ -133,62 +133,80 @@ public void valuePayloadWithSchemaAsJsonNode() { assertFalse(deserializedSchema.has("schema")); } + private void assertPrimitiveTemporalValues(IcebergChangeEvent event) { + Schema schema = event.icebergSchema(); + + Types.NestedField ship_date = schema.findField("ship_date"); + assertEquals(Types.IntegerType.get(), ship_date.type()); + assertEquals("io.debezium.time.Date", ship_date.doc()); + + Types.NestedField ship_timestamp = schema.findField("ship_timestamp"); + assertEquals(Types.LongType.get(), ship_timestamp.type()); + assertEquals("io.debezium.time.MicroTimestamp", ship_timestamp.doc()); + + Types.NestedField ship_timestamp_zoned = schema.findField("ship_timestamp_zoned"); + assertEquals(Types.StringType.get(), ship_timestamp_zoned.type()); + assertEquals("io.debezium.time.ZonedTimestamp", ship_timestamp_zoned.doc()); + + Types.NestedField ship_time = schema.findField("ship_time"); + assertEquals(Types.LongType.get(), ship_time.type()); + assertEquals("io.debezium.time.MicroTime", ship_time.doc()); + + Types.NestedField ship_time_zoned = schema.findField("ship_time_zoned"); + assertEquals(Types.StringType.get(), ship_time_zoned.type()); + assertEquals("io.debezium.time.ZonedTime", ship_time_zoned.doc()); + + GenericRecord record = event.asIcebergRecord(schema); + assertEquals(record.getField("ship_date"), 77663); + assertEquals(record.getField("ship_timestamp"), 6710075456016196L); + assertEquals(record.getField("ship_timestamp_zoned"), "2023-04-11T20:32:46.821144Z"); + assertEquals(record.getField("ship_time"), 73966821144L); + assertEquals(record.getField("ship_time_zoned"), "20:32:46.821144Z"); + } + @Test - public void coerceDebeziumTimeTypesDefaultBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesDefaultBehavior() throws JsonProcessingException { - IcebergSinkConfiguration config = TestConfig.builder() - .withLocalCatalog(localWarehouseDir) - .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); - String schemaString = schema.toString(); - String recordString = record.toString(); + IcebergChangeEvent event = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + this.defaultConfiguration + ); - assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); - assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); - assertTrue(recordString.contains("77663")); - assertTrue(recordString.contains("6710075456016196")); + assertPrimitiveTemporalValues(event); } @Test - public void coerceDebeziumTimeTypesDisabledBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesDisabledBehavior(@TempDir Path localWarehouseDir) throws JsonProcessingException { IcebergSinkConfiguration config = TestConfig.builder() .withLocalCatalog(localWarehouseDir) - .withCustomProperty("coerce.debezium-date", "false") - .withCustomProperty("coerce.debezium-micro-timestamp", "false") + .withCustomProperty("rich-temporal-types", "false") .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); - String schemaString = schema.toString(); - String recordString = record.toString(); + IcebergChangeEvent event = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + config + ); - assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); - assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); - assertTrue(recordString.contains("77663")); - assertTrue(recordString.contains("6710075456016196")); + assertPrimitiveTemporalValues(event); } @Test - public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWarehouseDir) throws JsonProcessingException { - IcebergSinkConfiguration config = TestConfig.builder() + IcebergSinkConfiguration configuration = TestConfig.builder() .withLocalCatalog(localWarehouseDir) - .withCustomProperty("coerce.debezium-date", "true") - .withCustomProperty("coerce.debezium-micro-timestamp", "true") + .withCustomProperty("rich-temporal-types", "true") .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + IcebergChangeEvent e = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + configuration + ); Schema schema = e.icebergSchema(); Types.NestedField ship_date = schema.findField("ship_date"); @@ -199,9 +217,24 @@ public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseD assertEquals(ship_timestamp.type(), Types.TimestampType.withoutZone()); assertEquals(ship_timestamp.doc(), "io.debezium.time.MicroTimestamp"); + Types.NestedField ship_timestamp_zoned = schema.findField("ship_timestamp_zoned"); + assertEquals(ship_timestamp_zoned.type(), Types.TimestampType.withZone()); + assertEquals(ship_timestamp_zoned.doc(), "io.debezium.time.ZonedTimestamp"); + + Types.NestedField ship_time = schema.findField("ship_time"); + assertEquals(ship_time.type(), Types.TimeType.get()); + assertEquals(ship_time.doc(), "io.debezium.time.MicroTime"); + + Types.NestedField ship_time_zoned = schema.findField("ship_time_zoned"); + assertEquals(ship_time_zoned.type(), Types.TimeType.get()); + assertEquals(ship_time_zoned.doc(), "io.debezium.time.ZonedTime"); + GenericRecord record = e.asIcebergRecord(schema); assertEquals(record.getField("ship_date"), LocalDate.parse("2182-08-20")); assertEquals(record.getField("ship_timestamp"), LocalDateTime.parse("2182-08-19T21:50:56.016196")); + assertEquals(record.getField("ship_timestamp_zoned"), ZonedDateTime.parse("2023-04-11T20:32:46.821144Z")); + assertEquals(record.getField("ship_time"), LocalTime.ofNanoOfDay(73966821144L * 1000)); + assertEquals(record.getField("ship_time_zoned"), OffsetTime.parse("20:32:46.821144Z").toLocalTime()); } @Test diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/IcebergChangeEventBuilder.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/IcebergChangeEventBuilder.java index e1a932c..82ccb20 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/IcebergChangeEventBuilder.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/IcebergChangeEventBuilder.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; +import com.getindata.kafka.connect.iceberg.sink.IcebergSinkConfiguration; +import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -28,6 +30,8 @@ public class IcebergChangeEventBuilder { ObjectNode keyPayload = JsonNodeFactory.instance.objectNode(); String destination = "test"; + private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); + public IcebergChangeEventBuilder() { } @@ -104,8 +108,8 @@ public IcebergChangeEvent build() { payload, keyPayload, this.valueSchema(), - this.keySchema() - ); + this.keySchema(), + this.defaultConfiguration); } private ObjectNode valueSchema() { diff --git a/src/test/resources/json/debezium-annotated-schema.json b/src/test/resources/json/debezium-annotated-schema.json index 1e3032d..bcfc40f 100644 --- a/src/test/resources/json/debezium-annotated-schema.json +++ b/src/test/resources/json/debezium-annotated-schema.json @@ -19,6 +19,28 @@ "optional": true, "name": "io.debezium.time.MicroTimestamp", "field": "ship_timestamp" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTimestamp", + "version": 1, + "field": "ship_timestamp_zoned" + }, + { + "type": "int64", + "optional": true, + "name": "io.debezium.time.MicroTime", + "version": 1, + "default": 0, + "field": "ship_time" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.time.ZonedTime", + "version": 1, + "field": "ship_time_zoned" } ], "optional": false, @@ -28,6 +50,9 @@ "id": 10003, "ship_date": 77663, "ship_timestamp": 6710075456016196, + "ship_timestamp_zoned": "2023-04-11T20:32:46.821144Z", + "ship_time": 73966821144, + "ship_time_zoned": "20:32:46.821144Z", "__op": "r", "__table": "time", "__lsn": 33832960, From ab09fa4567cd7b4c2cc36dae37511da00709e83e Mon Sep 17 00:00:00 2001 From: dacreify Date: Wed, 12 Apr 2023 14:15:26 -0700 Subject: [PATCH 14/50] give iceberg the zoned timestamp type it wants --- .../kafka/connect/iceberg/sink/IcebergChangeEvent.java | 2 +- .../getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 2e038a3..9b3c67f 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -181,7 +181,7 @@ private Object jsonValToIcebergVal(Types.NestedField field, break; case TIMESTAMP: if (node.isTextual()) { - val = ZonedDateTime.parse(node.asText()); + val = OffsetDateTime.parse(node.asText()); } else if (node.isNumber()) { Instant instant = Instant.ofEpochSecond(0L, node.asLong() * 1000); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 2e0a1dc..ecbbfe0 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -232,7 +232,7 @@ public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWareho GenericRecord record = e.asIcebergRecord(schema); assertEquals(record.getField("ship_date"), LocalDate.parse("2182-08-20")); assertEquals(record.getField("ship_timestamp"), LocalDateTime.parse("2182-08-19T21:50:56.016196")); - assertEquals(record.getField("ship_timestamp_zoned"), ZonedDateTime.parse("2023-04-11T20:32:46.821144Z")); + assertEquals(record.getField("ship_timestamp_zoned"), OffsetDateTime.parse("2023-04-11T20:32:46.821144Z")); assertEquals(record.getField("ship_time"), LocalTime.ofNanoOfDay(73966821144L * 1000)); assertEquals(record.getField("ship_time_zoned"), OffsetTime.parse("20:32:46.821144Z").toLocalTime()); } From d368fbc61324101efc9536fc17abcc6090aea791 Mon Sep 17 00:00:00 2001 From: dacreify Date: Wed, 12 Apr 2023 14:24:04 -0700 Subject: [PATCH 15/50] style, changelog update --- CHANGELOG.md | 3 ++- .../SinkRecordToIcebergChangeEventConverter.java | 3 ++- .../connect/iceberg/sink/TestIcebergUtil.java | 15 ++++++++++----- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d849e1f..9549658 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,7 +2,8 @@ ## [Unreleased] -- Added support for coercion of two Debezium time types: Date and MicroTimestamp +- Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime +- Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` ## [0.2.5] - 2023-03-20 diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java index e35ea64..6e8c23f 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/converter/SinkRecordToIcebergChangeEventConverter.java @@ -26,7 +26,8 @@ public SinkRecordToIcebergChangeEventConverter(Transformation extrac JsonConverter keyJsonConverter, JsonConverter valueJsonConverter, Deserializer keyDeserializer, - Deserializer valueDeserializer, IcebergSinkConfiguration configuration) { + Deserializer valueDeserializer, + IcebergSinkConfiguration configuration) { this.extractNewRecordStateTransformation = extractNewRecordStateTransformation; this.keyJsonConverter = keyJsonConverter; this.valueJsonConverter = valueJsonConverter; diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index ecbbfe0..d6d6486 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -50,7 +50,8 @@ class TestIcebergUtil { public void testNestedJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(serdeWithSchema).get("payload"), null, - MAPPER.readTree(serdeWithSchema).get("schema"), null, this.defaultConfiguration); + MAPPER.readTree(serdeWithSchema).get("schema"), null, + this.defaultConfiguration); Schema schema = e.icebergSchema(); assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + "4:")); @@ -60,7 +61,8 @@ public void testNestedJsonRecord() throws JsonProcessingException { public void testUnwrapJsonRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithSchema).get("payload"), null, - MAPPER.readTree(unwrapWithSchema).get("schema"), null, this.defaultConfiguration); + MAPPER.readTree(unwrapWithSchema).get("schema"), null, + this.defaultConfiguration); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); assertEquals("orders", record.getField("__table").toString()); @@ -71,7 +73,8 @@ public void testUnwrapJsonRecord() throws IOException { public void testNestedArrayJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, this.defaultConfiguration); + MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, + this.defaultConfiguration); Schema schema = e.icebergSchema(); assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); @@ -87,7 +90,8 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { assertThrows(RuntimeException.class, () -> { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema2).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null, this.defaultConfiguration); + MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null, + this.defaultConfiguration); Schema schema = e.icebergSchema(); System.out.println(schema.asStruct()); System.out.println(schema); @@ -100,7 +104,8 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { public void testNestedGeomJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithGeomSchema).get("payload"), null, - MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, this.defaultConfiguration); + MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, + this.defaultConfiguration); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); From 3d711692b8fad5004c974876d9a0d8e318c29d31 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 7 Apr 2023 14:10:41 -0700 Subject: [PATCH 16/50] Added support for Array types containing nested Struct element types such as "io.debezium.connector.common.TransactionMetadataValue" Added to support metadata via debezium connector config: "provide.transaction.metadata": "true" --- .../iceberg/sink/IcebergChangeEvent.java | 39 +++++++++-- .../connect/iceberg/sink/TestIcebergUtil.java | 32 +++++++++ .../json/debezium-metadata-schema.json | 65 +++++++++++++++++++ 3 files changed, 130 insertions(+), 6 deletions(-) create mode 100644 src/test/resources/json/debezium-metadata-schema.json diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 9b3c67f..87ad7ac 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -15,6 +15,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Type.TypeID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -223,7 +224,19 @@ else if (node.isNull()){ } break; case LIST: - val = MAPPER.convertValue(node, ArrayList.class); + // for now we support two LIST type cases + Types.ListType listType = (Types.ListType) field.type(); + if (listType.elementType().typeId() == TypeID.STRUCT) { + List structList = new ArrayList<>(); + Iterator it = node.iterator(); + while (it.hasNext()) { + structList.add(asIcebergRecord(listType.elementType().asStructType(), it.next())); + } + val = structList; + } + else { + val = MAPPER.convertValue(node, ArrayList.class); + } break; case MAP: val = MAPPER.convertValue(node, Map.class); @@ -358,14 +371,28 @@ private List icebergSchema(JsonNode eventSchema, String schem if (items != null && items.has("type")) { String listItemType = items.get("type").textValue(); - if (listItemType.equals("struct") || listItemType.equals("array") || listItemType.equals("map")) { - throw new RuntimeException("Complex nested array types are not supported," + + + if (listItemType.equals("array") || listItemType.equals("map")) { + throw new RuntimeException("'array' and 'map' nested array types are not supported," + " array[" + listItemType + "], field " + fieldName); } + else { + if (listItemType.equals("struct")) { + List subSchema = icebergSchema(items, fieldName, columnId+2); + schemaColumns.add(Types.NestedField.optional(columnId, + fieldName, + Types.ListType.ofOptional(columnId+1, + Types.StructType.of(subSchema)), + "")); + columnId += subSchema.size() + 2; + } + else { // primitive coercions are not supported for list types, pass '""' for fieldTypeName - Type.PrimitiveType item = icebergFieldType(listItemType, ""); - schemaColumns.add(Types.NestedField.optional( - columnId, fieldName, Types.ListType.ofOptional(++columnId, item), "")); + Type.PrimitiveType item = icebergFieldType(listItemType, ""); + schemaColumns.add(Types.NestedField.optional( + columnId, fieldName, Types.ListType.ofOptional(++columnId, item), "")); + } + } } else { throw new RuntimeException("Unexpected Array type for field " + fieldName); } diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index d6d6486..9e06145 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -31,6 +31,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.ArrayList; import java.util.Set; import static org.junit.jupiter.api.Assertions.*; @@ -43,6 +44,7 @@ class TestIcebergUtil { final String unwrapWithArraySchema = Testing.Files.readResourceAsString("json/serde-with-array.json"); final String unwrapWithArraySchema2 = Testing.Files.readResourceAsString("json/serde-with-array2.json"); final String debeziumTimeCoercionSchema = Testing.Files.readResourceAsString("json/debezium-annotated-schema.json"); + final String debeziumMetadataSchema = Testing.Files.readResourceAsString("json/debezium-metadata-schema.json"); private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); @@ -242,6 +244,36 @@ public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWareho assertEquals(record.getField("ship_time_zoned"), OffsetTime.parse("20:32:46.821144Z").toLocalTime()); } + @Test + public void listStructSchemaHandling() + throws JsonProcessingException { + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumMetadataSchema).get("payload"), null, + MAPPER.readTree(debeziumMetadataSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + String schemaString = schema.toString(); + + GenericRecord record = e.asIcebergRecord(schema); + + assertTrue(schemaString.contains("data_collections: optional list Date: Thu, 13 Apr 2023 09:59:20 -0700 Subject: [PATCH 17/50] updated CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9549658..8fa5fff 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ ## [Unreleased] +- Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` From 6382ae2755ee35f461ea3950298eb1f57832345a Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 13 Apr 2023 10:02:57 -0700 Subject: [PATCH 18/50] unit test reparations --- .../getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 9e06145..7653995 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -249,7 +249,9 @@ public void listStructSchemaHandling() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(debeziumMetadataSchema).get("payload"), null, - MAPPER.readTree(debeziumMetadataSchema).get("schema"), null); + MAPPER.readTree(debeziumMetadataSchema).get("schema"), null, + defaultConfiguration + ); Schema schema = e.icebergSchema(); String schemaString = schema.toString(); From 4921ac4d5a60b9fe301e64103dc3c6d7bd7d9af3 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 13 Apr 2023 13:16:02 -0700 Subject: [PATCH 19/50] unit test adjs. format adjs. --- .../connect/iceberg/sink/TestIcebergUtil.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 7653995..da94e4c 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -248,9 +248,9 @@ public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWareho public void listStructSchemaHandling() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumMetadataSchema).get("payload"), null, - MAPPER.readTree(debeziumMetadataSchema).get("schema"), null, - defaultConfiguration + MAPPER.readTree(debeziumMetadataSchema).get("payload"), null, + MAPPER.readTree(debeziumMetadataSchema).get("schema"), null, + defaultConfiguration ); Schema schema = e.icebergSchema(); String schemaString = schema.toString(); @@ -261,19 +261,19 @@ public void listStructSchemaHandling() GenericRecord innerRecord = (GenericRecord) ((ArrayList) record.getField("data_collections")).get(0); Object value = innerRecord.getField("data_collection"); - assertTrue(value.equals("public.mine")); + assertEquals("public.mine", value); value = innerRecord.getField("event_count"); - assertTrue((long) value == 1); + assertEquals(1L, value); value = record.getField("status"); - assertTrue(((String) value).equals("END")); + assertEquals("END", value); value = record.getField("id"); - assertTrue(((String) value).equals("12117:67299632")); + assertEquals("12117:67299632", value); value = record.getField("ts_ms"); - assertTrue(((long) value) == 1680821545908L); + assertEquals(1680821545908L, value); } @Test From 8500b991a57e17c3f42cc8affa084f1cfc67f0f1 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 13 Apr 2023 14:26:29 -0700 Subject: [PATCH 20/50] update aws lakeformation transitive dependency for sink to provide lakeformation support in s3 iceberg tables --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 93d6bb1..6020715 100644 --- a/pom.xml +++ b/pom.xml @@ -71,6 +71,12 @@ provided + + software.amazon.awssdk + lakeformation + 2.20.45 + + org.apache.iceberg iceberg-spark-runtime-3.2_2.13 From 107e50ef7deaaad798b15f2ae0821f839bd0e673 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 13 Apr 2023 14:29:22 -0700 Subject: [PATCH 21/50] updated CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9549658..68d5339 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,7 @@ # Changelog +- Updates AWS lakeformation transitive dependency providing lakeformation support in s3 iceberg tables. + ## [Unreleased] - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime From c06be5c4db94cdb3d257fbfe374637844931959f Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 14 Apr 2023 13:48:19 -0700 Subject: [PATCH 22/50] update to iceberg 1.2.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6020715..5d4bd66 100644 --- a/pom.xml +++ b/pom.xml @@ -15,7 +15,7 @@ 3.2.2 - 1.0.0 + 1.2.1 1.9.7.Final 3.3.3 2.17.295 From c1cb8599b458e8c85132263f495dbe972896b5d9 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Mon, 24 Apr 2023 15:02:06 -0700 Subject: [PATCH 23/50] Updated hadoop.version to '3.3.5' and amazon-sdk to '2.20.51' for CVE repair. --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 5d4bd66..203141a 100644 --- a/pom.xml +++ b/pom.xml @@ -17,8 +17,8 @@ 3.2.2 1.2.1 1.9.7.Final - 3.3.3 - 2.17.295 + 3.3.5 + 2.20.51 3.1.3 From ad09e4140fcb1025560c7691540fb9f5caf1db18 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Tue, 25 Apr 2023 09:49:52 -0700 Subject: [PATCH 24/50] updated transitive dependency for parquet-hadoop-bundle to '1.13.0' --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 203141a..f253df1 100644 --- a/pom.xml +++ b/pom.xml @@ -89,6 +89,12 @@ ${debezium.version} + + org.apache.parquet + parquet-hadoop-bundle + 1.13.0 + + org.apache.hadoop hadoop-common From cee263456bbd2011665f9b5a2cd7d79a66382d0f Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Tue, 25 Apr 2023 10:12:43 -0700 Subject: [PATCH 25/50] updated CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 96bd576..58a4fee 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,9 @@ # Changelog -- Updates AWS lakeformation transitive dependency providing lakeformation support in s3 iceberg tables. - ## [Unreleased] +- Updates dependencies to resolve some jackson-databind critical CVEs. +- Updates AWS lakeformation transitive dependency providing lakeformation support in s3 iceberg tables. - Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` From 0e169d5664b3b799f37e185decf79f936b532e01 Mon Sep 17 00:00:00 2001 From: Johan Henriksson Date: Mon, 20 Mar 2023 19:13:12 +0100 Subject: [PATCH 26/50] add partitioning column configuration setting --- .../kafka/connect/iceberg/sink/IcebergChangeEvent.java | 8 ++++---- .../connect/iceberg/sink/IcebergSinkConfiguration.java | 9 ++++++++- .../iceberg/sink/tableoperator/IcebergTableOperator.java | 2 +- .../kafka/connect/iceberg/sink/TestIcebergUtil.java | 6 +++--- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 87ad7ac..1a1aebc 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -68,12 +68,12 @@ public String destinationTable() { return destination.replace(".", "_").replace("-", "_"); } - public GenericRecord asIcebergRecord(Schema schema) { + public GenericRecord asIcebergRecord(Schema schema, String partitionColumn) { final GenericRecord record = asIcebergRecord(schema.asStruct(), value); - if (value != null && value.has("__source_ts_ms") && value.get("__source_ts_ms") != null) { - final long source_ts_ms = value.get("__source_ts_ms").longValue(); - final OffsetDateTime odt = OffsetDateTime.ofInstant(Instant.ofEpochMilli(source_ts_ms), ZoneOffset.UTC); + if (value != null && value.has(partitionColumn) && value.get(partitionColumn) != null) { + final long partitionTimestamp = value.get(partitionColumn).longValue(); + final OffsetDateTime odt = OffsetDateTime.ofInstant(Instant.ofEpochMilli(partitionTimestamp), ZoneOffset.UTC); record.setField("__source_ts", odt); } else { record.setField("__source_ts", null); diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index f871d74..a156287 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -27,6 +27,8 @@ public class IcebergSinkConfiguration { public static final String CATALOG_NAME = ICEBERG_PREFIX + "name"; public static final String CATALOG_IMPL = ICEBERG_PREFIX + "catalog-impl"; public static final String CATALOG_TYPE = ICEBERG_PREFIX + "type"; + public static final String PARTITION_COLUMN = ICEBERG_PREFIX + "partition"; + private static final ConfigDef CONFIG_DEF = new ConfigDef() .define(UPSERT, BOOLEAN, true, MEDIUM, "When true Iceberg rows will be updated based on table primary key. " + @@ -60,8 +62,9 @@ public class IcebergSinkConfiguration { .define(CATALOG_TYPE, STRING, null, MEDIUM, "Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type " + "can be set to non null value at the same time)") + .define(PARTITION_COLUMN, STRING, "__source_ts_ms", MEDIUM, + "Column used for partitioning. Must be unix millisecond timestamp.") ; - private final AbstractConfig parsedConfig; private final Map properties; @@ -114,6 +117,10 @@ public String getCatalogName() { return parsedConfig.getString(CATALOG_NAME); } + public String getPartitionColumn() { + return parsedConfig.getString(PARTITION_COLUMN); + } + public Map getIcebergCatalogConfiguration() { return getConfiguration(ICEBERG_PREFIX); } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java index 91d8542..183f5ca 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java @@ -160,7 +160,7 @@ private void addToTablePerSchema(Table icebergTable, List ev BaseTaskWriter writer = writerFactory.create(icebergTable); try { for (IcebergChangeEvent e : events) { - writer.write(e.asIcebergRecord(icebergTable.schema())); + writer.write(e.asIcebergRecord(icebergTable.schema(), configuration.getPartitionColumn())); } writer.close(); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index da94e4c..da89d09 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -66,7 +66,7 @@ public void testUnwrapJsonRecord() throws IOException { MAPPER.readTree(unwrapWithSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); + GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); assertEquals("orders", record.getField("__table").toString()); assertEquals(16850, record.getField("order_date")); } @@ -83,7 +83,7 @@ public void testNestedArrayJsonRecord() throws JsonProcessingException { System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); assertEquals(schema.findField("schedule").type().asListType().elementType().toString(), "string"); - GenericRecord record = e.asIcebergRecord(schema); + GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); assertTrue(record.toString().contains("[10000, 10001, 10002, 10003]")); } @@ -109,7 +109,7 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); + GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); GenericRecord g = (GenericRecord) record.getField("g"); GenericRecord h = (GenericRecord) record.getField("h"); From a6d3127a1e9297a2820338eb2cf44f9b356bbbf2 Mon Sep 17 00:00:00 2001 From: Johan Henriksson Date: Mon, 20 Mar 2023 19:58:32 +0100 Subject: [PATCH 27/50] add custom partition column test --- .../connect/iceberg/sink/TestIcebergUtil.java | 15 +++++++++++++ .../json/custom-partition-column.json | 22 +++++++++++++++++++ 2 files changed, 37 insertions(+) create mode 100644 src/test/resources/json/custom-partition-column.json diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index da89d09..9f66ebb 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -48,6 +48,8 @@ class TestIcebergUtil { private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); + final String customPartitionColumn = Testing.Files.readResourceAsString("json/custom-partition-column.json"); + @Test public void testNestedJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", @@ -119,6 +121,19 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { assertNull(h.get(0, Types.BinaryType.get().typeId().javaClass())); } + @Test + public void testCustomPartitionColumnRecord() throws IOException { + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(customPartitionColumn).get("payload"), null, + MAPPER.readTree(customPartitionColumn).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema, "timestamp"); + assertEquals("2023-03-20T18:25:27.865Z", record.getField("__source_ts").toString()); + assertEquals("hello", record.getField("message")); + System.out.println(schema); + System.out.println(record); + } + @Test public void valuePayloadWithSchemaAsJsonNode() { // testing Debezium deserializer diff --git a/src/test/resources/json/custom-partition-column.json b/src/test/resources/json/custom-partition-column.json new file mode 100644 index 0000000..817b6ca --- /dev/null +++ b/src/test/resources/json/custom-partition-column.json @@ -0,0 +1,22 @@ +{ + "schema": { + "type": "struct", + "optional": false, + "fields": [ + { + "type": "string", + "optional": false, + "field": "message" + }, + { + "type": "int32", + "optional": false, + "field": "timestamp" + } + ] + }, + "payload": { + "message": "hello", + "timestamp": 1679336727865 + } +} \ No newline at end of file From 63ae68afa998457ea63d33212b249f7a5749b270 Mon Sep 17 00:00:00 2001 From: Johan Henriksson Date: Mon, 20 Mar 2023 19:59:10 +0100 Subject: [PATCH 28/50] add custom partition column config to readme & update changelog --- CHANGELOG.md | 1 + README.md | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 58a4fee..266125d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ - Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` +- Add `iceberg.partition` config setting to allow any column to be used for partitioning. ## [0.2.5] - 2023-03-20 diff --git a/README.md b/README.md index 955ebb4..0f217df 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ mvn clean package | iceberg.type | String | *null* | Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time) | | iceberg.* | | | All properties with this prefix will be passed to Iceberg Catalog implementation | | iceberg.table-default.* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | - +| iceberg.partition | String | __source_ts_ms | Column used for partitioning. Must be a UTC unix millisecond timestamp. | ### REST / Manual based installation From 3622453863bdb4156d270711d892545ef2fed5b0 Mon Sep 17 00:00:00 2001 From: Johan Henriksson Date: Tue, 21 Mar 2023 16:44:21 +0100 Subject: [PATCH 29/50] add iceberg.partition.column and iceberg.partition.timestamp config --- README.md | 45 ++++++++------- .../iceberg/sink/IcebergChangeConsumer.java | 2 +- .../iceberg/sink/IcebergChangeEvent.java | 45 +++++++++------ .../sink/IcebergSinkConfiguration.java | 22 +++++++- .../connect/iceberg/sink/IcebergUtil.java | 4 +- .../tableoperator/IcebergTableOperator.java | 6 +- .../iceberg/sink/IcebergSinkSystemTest.java | 8 ++- .../connect/iceberg/sink/TestIcebergUtil.java | 56 +++++++++++-------- .../IcebergTableOperatorTest.java | 2 +- 9 files changed, 120 insertions(+), 70 deletions(-) diff --git a/README.md b/README.md index 0f217df..17e21f4 100644 --- a/README.md +++ b/README.md @@ -12,22 +12,23 @@ mvn clean package ### Configuration reference -| Key | Type | Default value | Description | -|-------------------------|---------|----------------|----------------------------------------------------------------------------------------------------------------------------------------| -| upsert | boolean | true | When *true* Iceberg rows will be updated based on table primary key. When *false* all modification will be added as separate rows. | -| upsert.keep-deletes | boolean | true | When *true* delete operation will leave a tombstone that will have only a primary key and *__deleted** flag set to true | -| upsert.dedup-column | String | __source_ts_ms | Column used to check which state is newer during upsert | -| upsert.op-column | String | __op | Column used to check which state is newer during upsert when *upsert.dedup-column* is not enough to resolve | -| allow-field-addition | boolean | true | When *true* sink will be adding new columns to Iceberg tables on schema changes | -| table.auto-create | boolean | false | When *true* sink will automatically create new Iceberg tables | -| table.namespace | String | default | Table namespace. In Glue it will be used as database name | -| table.prefix | String | *empty string* | Prefix added to all table names | -| iceberg.name | String | default | Iceberg catalog name | -| iceberg.catalog-impl | String | *null* | Iceberg catalog implementation (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time | -| iceberg.type | String | *null* | Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time) | -| iceberg.* | | | All properties with this prefix will be passed to Iceberg Catalog implementation | -| iceberg.table-default.* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | -| iceberg.partition | String | __source_ts_ms | Column used for partitioning. Must be a UTC unix millisecond timestamp. | +| Key | Type | Default value | Description | +|-----------------------------|---------|----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| +| upsert | boolean | true | When *true* Iceberg rows will be updated based on table primary key. When *false* all modification will be added as separate rows. | +| upsert.keep-deletes | boolean | true | When *true* delete operation will leave a tombstone that will have only a primary key and *__deleted** flag set to true | +| upsert.dedup-column | String | __source_ts_ms | Column used to check which state is newer during upsert | +| upsert.op-column | String | __op | Column used to check which state is newer during upsert when *upsert.dedup-column* is not enough to resolve | +| allow-field-addition | boolean | true | When *true* sink will be adding new columns to Iceberg tables on schema changes | +| table.auto-create | boolean | false | When *true* sink will automatically create new Iceberg tables | +| table.namespace | String | default | Table namespace. In Glue it will be used as database name | +| table.prefix | String | *empty string* | Prefix added to all table names | +| iceberg.name | String | default | Iceberg catalog name | +| iceberg.catalog-impl | String | *null* | Iceberg catalog implementation (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time | +| iceberg.type | String | *null* | Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time) | +| iceberg.* | | | All properties with this prefix will be passed to Iceberg Catalog implementation | +| iceberg.table-default.* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | +| iceberg.partition.column | String | __source_ts | Column used for partitioning. If the column already exists, it must be of type timestamp. | +| iceberg.partition.timestamp | String | __source_ts_ms | Column containing unix millisecond timestamps to be converted to partitioning times. If equal to partition.column, values will be replaced with timestamps. | ### REST / Manual based installation @@ -302,9 +303,15 @@ Rows cannot be updated nor removed unless primary key is defined. In case of del ### Iceberg partitioning support -Currently, partitioning is done automatically based on event time. Partitioning only works when Debezium is configured in append-only mode (`upsert: false`). +The consumer reads unix millisecond timestamps from the event field configured in `iceberg.partition.timestamp`, converts them to iceberg +timestamps, and writes them to the table column configured in `iceberg.partition.column`. The timestamp column is then used to extract a +date to be used as the partitioning key. If `iceberg.partition.timestamp` is empty, `iceberg.parition.column` is assumed to already be of +type timestamp, and no conversion is performed. If they are set to the same value, the integer values will be replaced by the converted +timestamp values. -Any event produced by debezium source contains a source time at which the transaction was committed: +Partitioning only works when configured in append-only mode (`upsert: false`). + +By default, the sink expects to receive events produced by a debezium source containing a source time at which the transaction was committed: ```sql "sourceOffset": { @@ -313,8 +320,6 @@ Any event produced by debezium source contains a source time at which the transa } ``` -From this value day part is extracted and used as partition. - ## Debezium change event format support Kafka Connect Iceberg Sink is expecting events in a format of *Debezium change event*. It uses however only an *after* portion of that event and some metadata. diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeConsumer.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeConsumer.java index 9173922..47e3164 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeConsumer.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeConsumer.java @@ -65,7 +65,7 @@ private Table loadIcebergTable(Catalog icebergCatalog, TableIdentifier tableId, if (!configuration.isTableAutoCreate()) { throw new ConnectException(String.format("Table '%s' not found! Set '%s' to true to create tables automatically!", tableId, TABLE_AUTO_CREATE)); } - return IcebergUtil.createIcebergTable(icebergCatalog, tableId, sampleEvent.icebergSchema(), configuration); + return IcebergUtil.createIcebergTable(icebergCatalog, tableId, sampleEvent.icebergSchema(configuration.getPartitionColumn()), configuration); }); } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 1a1aebc..cf52ac5 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -60,23 +60,26 @@ public JsonSchema jsonSchema() { return jsonSchema; } - public Schema icebergSchema() { - return jsonSchema.icebergSchema(); + public Schema icebergSchema(String partitionColumn) { + return jsonSchema.icebergSchema(partitionColumn); } public String destinationTable() { return destination.replace(".", "_").replace("-", "_"); } - public GenericRecord asIcebergRecord(Schema schema, String partitionColumn) { + public GenericRecord asIcebergRecord(Schema schema, String partitionColumn, String partitionTimestampColumn) { final GenericRecord record = asIcebergRecord(schema.asStruct(), value); - if (value != null && value.has(partitionColumn) && value.get(partitionColumn) != null) { - final long partitionTimestamp = value.get(partitionColumn).longValue(); - final OffsetDateTime odt = OffsetDateTime.ofInstant(Instant.ofEpochMilli(partitionTimestamp), ZoneOffset.UTC); - record.setField("__source_ts", odt); - } else { - record.setField("__source_ts", null); + if (partitionTimestampColumn != null && !partitionTimestampColumn.equals("")) { + // if partitionTimestampColumn is set, convert it to a timestamp and store it in partitionColumn. + if (value != null && value.has(partitionTimestampColumn) && value.get(partitionTimestampColumn) != null) { + final long partitionTimestamp = value.get(partitionTimestampColumn).longValue(); + final OffsetDateTime odt = OffsetDateTime.ofInstant(Instant.ofEpochMilli(partitionTimestamp), ZoneOffset.UTC); + record.setField(partitionColumn, odt); + } else { + record.setField(partitionColumn, null); + } } return record; } @@ -146,6 +149,8 @@ else if (configuration.isRichTemporalTypes() && } case "bytes": return Types.BinaryType.get(); + case "timestamptz": + return Types.TimestampType.withZone(); default: // default to String type return Types.StringType.get(); @@ -296,22 +301,22 @@ private List KeySchemaFields() { return new ArrayList<>(); } - private List valueSchemaFields() { + private List valueSchemaFields(String partitionColumn) { if (valueSchema != null && valueSchema.has("fields") && valueSchema.get("fields").isArray()) { LOGGER.debug(valueSchema.toString()); - return icebergSchema(valueSchema, "", 0, true); + return icebergSchema(valueSchema, "", 0, true, partitionColumn); } LOGGER.trace("Event schema not found!"); return new ArrayList<>(); } - public Schema icebergSchema() { + public Schema icebergSchema(String partitionColumn) { if (this.valueSchema == null) { throw new RuntimeException("Failed to get event schema, event schema is null"); } - final List tableColumns = valueSchemaFields(); + final List tableColumns = valueSchemaFields(partitionColumn); if (tableColumns.isEmpty()) { throw new RuntimeException("Failed to get event schema, event schema has no fields!"); @@ -345,11 +350,11 @@ public Schema icebergSchema() { } private List icebergSchema(JsonNode eventSchema, String schemaName, int columnId) { - return icebergSchema(eventSchema, schemaName, columnId, false); + return icebergSchema(eventSchema, schemaName, columnId, false, ""); } private List icebergSchema(JsonNode eventSchema, String schemaName, int columnId, - boolean addSourceTsField) { + boolean addPartitionField, String partitionColumn) { List schemaColumns = new ArrayList<>(); String schemaType = eventSchema.get("type").textValue(); LOGGER.debug("Converting Schema of: {}::{}", schemaName, schemaType); @@ -410,6 +415,12 @@ private List icebergSchema(JsonNode eventSchema, String schem columnId += subSchema.size(); break; default: //primitive types + if (fieldName.equals(partitionColumn)) { + // if it is the partition column, swap its type to timestamp + fieldType = "timestamptz"; + // we also dont need to add a partition field, since it already exists. + addPartitionField = false; + } // passing fieldTypeName for NestedField `doc` attribute, // annotation based value coercions can be made utilizing the NestedField `doc` initializer/method schemaColumns.add(Types.NestedField.optional(columnId, fieldName, @@ -419,9 +430,9 @@ private List icebergSchema(JsonNode eventSchema, String schem } } - if (addSourceTsField) { + if (addPartitionField) { columnId++; - schemaColumns.add(Types.NestedField.optional(columnId, "__source_ts", + schemaColumns.add(Types.NestedField.optional(columnId, partitionColumn, Types.TimestampType.withZone(), "")); } return schemaColumns; diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index a156287..fe50fcf 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -27,7 +27,8 @@ public class IcebergSinkConfiguration { public static final String CATALOG_NAME = ICEBERG_PREFIX + "name"; public static final String CATALOG_IMPL = ICEBERG_PREFIX + "catalog-impl"; public static final String CATALOG_TYPE = ICEBERG_PREFIX + "type"; - public static final String PARTITION_COLUMN = ICEBERG_PREFIX + "partition"; + public static final String PARTITION_TIMESTAMP = ICEBERG_PREFIX + "partition.timestamp"; + public static final String PARTITION_COLUMN = ICEBERG_PREFIX + "partition.column"; private static final ConfigDef CONFIG_DEF = new ConfigDef() .define(UPSERT, BOOLEAN, true, MEDIUM, @@ -62,8 +63,11 @@ public class IcebergSinkConfiguration { .define(CATALOG_TYPE, STRING, null, MEDIUM, "Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type " + "can be set to non null value at the same time)") - .define(PARTITION_COLUMN, STRING, "__source_ts_ms", MEDIUM, - "Column used for partitioning. Must be unix millisecond timestamp.") + .define(PARTITION_TIMESTAMP, STRING, "__source_ts_ms", MEDIUM, + "Unix millisecond timestamp used to fill the partitioning column. If set, values"+ + "will be converted to a timestamp value and stored in iceberg.partition.column") + .define(PARTITION_COLUMN, STRING, "__source_ts", MEDIUM, + "Column used for partitioning. If the column already exists, it must be of type timestamp.") ; private final AbstractConfig parsedConfig; private final Map properties; @@ -117,10 +121,22 @@ public String getCatalogName() { return parsedConfig.getString(CATALOG_NAME); } + /** + * Gets the name of the column used for partitioning the iceberg table. + * @return Name of the partitioning column + */ public String getPartitionColumn() { return parsedConfig.getString(PARTITION_COLUMN); } + /** + * Gets the name of the column containing unix millisecond timestamps to be used for partitioning. + * @return Unix timestamp in milliseconds + */ + public String getPartitionTimestamp() { + return parsedConfig.getString(PARTITION_TIMESTAMP); + } + public Map getIcebergCatalogConfiguration() { return getConfiguration(ICEBERG_PREFIX); } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java index d56a887..26af7c1 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java @@ -42,8 +42,8 @@ public static Table createIcebergTable(Catalog icebergCatalog, TableIdentifier t boolean partition = !configuration.isUpsert(); final PartitionSpec ps; - if (partition && schema.findField("__source_ts") != null) { - ps = PartitionSpec.builderFor(schema).day("__source_ts").build(); + if (partition && schema.findField(configuration.getPartitionColumn()) != null) { + ps = PartitionSpec.builderFor(schema).day(configuration.getPartitionColumn()).build(); } else { ps = PartitionSpec.builderFor(schema).build(); } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java index 183f5ca..0fa7554 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperator.java @@ -64,7 +64,7 @@ public void addToTable(Table icebergTable, List events) { for (Map.Entry> schemaEvents : eventsGroupedBySchema.entrySet()) { // extend table schema if new fields found - applyFieldAddition(icebergTable, schemaEvents.getKey().icebergSchema()); + applyFieldAddition(icebergTable, schemaEvents.getKey().icebergSchema(configuration.getPartitionColumn())); // add set of events to table addToTablePerSchema(icebergTable, schemaEvents.getValue()); } @@ -160,7 +160,9 @@ private void addToTablePerSchema(Table icebergTable, List ev BaseTaskWriter writer = writerFactory.create(icebergTable); try { for (IcebergChangeEvent e : events) { - writer.write(e.asIcebergRecord(icebergTable.schema(), configuration.getPartitionColumn())); + writer.write(e.asIcebergRecord(icebergTable.schema(), + configuration.getPartitionColumn(), + configuration.getPartitionTimestamp())); } writer.close(); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkSystemTest.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkSystemTest.java index e113a9a..9296113 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkSystemTest.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkSystemTest.java @@ -84,8 +84,12 @@ void shouldUpsertValue() throws SQLException { postgresTestHelper.execute("create table dbz_test1 (timestamp bigint, id int PRIMARY KEY, value varchar(20))"); postgresTestHelper.execute("insert into dbz_test1 values(123, 1, 'ABC')"); + var v = sparkTestHelper.query("SELECT 1").count(); + String query = "SELECT timestamp, id, value FROM " + getIcebergTableName("dbz_test1"); - given().ignoreExceptions().await().atMost(Duration.ofSeconds(15)).until(() -> sparkTestHelper.query(query).count() == 1); + given().ignoreExceptions().await().atMost(Duration.ofSeconds(1800)).until(() -> { + return sparkTestHelper.query(query).count() == 1; + }); Dataset result = sparkTestHelper.query(query); assertThat(result.count()).isEqualTo(1); @@ -118,7 +122,7 @@ void shouldModifySchema() throws SQLException { postgresTestHelper.execute("insert into dbz_test2 values(123, 1, 'ABC')"); String query1 = "SELECT timestamp, id, value FROM " + getIcebergTableName("dbz_test2"); - given().ignoreExceptions().await().atMost(Duration.ofSeconds(15)).until(() -> sparkTestHelper.query(query1).count() == 1); + given().ignoreExceptions().await().atMost(Duration.ofSeconds(150)).until(() -> sparkTestHelper.query(query1).count() == 1); postgresTestHelper.execute("alter table dbz_test2 add column value2 varchar(20)"); postgresTestHelper.execute("insert into dbz_test2 values(456, 2, 'DEF', 'XYZ')"); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 9f66ebb..bea72b2 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -38,6 +38,9 @@ class TestIcebergUtil { private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final String defaultPartitionTimestamp = "__source_ts_ms"; + private static final String defaultPartitionColumn = "__source_ts"; + final String serdeWithSchema = Testing.Files.readResourceAsString("json/serde-with-schema.json"); final String unwrapWithSchema = Testing.Files.readResourceAsString("json/unwrap-with-schema.json"); final String unwrapWithGeomSchema = Testing.Files.readResourceAsString("json/serde-with-schema_geom.json"); @@ -54,9 +57,8 @@ class TestIcebergUtil { public void testNestedJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(serdeWithSchema).get("payload"), null, - MAPPER.readTree(serdeWithSchema).get("schema"), null, - this.defaultConfiguration); - Schema schema = e.icebergSchema(); + MAPPER.readTree(serdeWithSchema).get("schema"), null, this.defaultConfiguration); + Schema schema = e.icebergSchema(defaultPartitionColumn); assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + "4:")); } @@ -65,10 +67,9 @@ public void testNestedJsonRecord() throws JsonProcessingException { public void testUnwrapJsonRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithSchema).get("payload"), null, - MAPPER.readTree(unwrapWithSchema).get("schema"), null, - this.defaultConfiguration); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); + MAPPER.readTree(unwrapWithSchema).get("schema"), null, this.defaultConfiguration); + Schema schema = e.icebergSchema(defaultPartitionColumn); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertEquals("orders", record.getField("__table").toString()); assertEquals(16850, record.getField("order_date")); } @@ -77,15 +78,15 @@ public void testUnwrapJsonRecord() throws IOException { public void testNestedArrayJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, - this.defaultConfiguration); - Schema schema = e.icebergSchema(); + MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, this.defaultConfiguration); + Schema schema = e.icebergSchema(defaultPartitionColumn); assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); + System.out.println(schema.asStruct()); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); assertEquals(schema.findField("schedule").type().asListType().elementType().toString(), "string"); - GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertTrue(record.toString().contains("[10000, 10001, 10002, 10003]")); } @@ -94,9 +95,8 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { assertThrows(RuntimeException.class, () -> { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithArraySchema2).get("payload"), null, - MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null, - this.defaultConfiguration); - Schema schema = e.icebergSchema(); + MAPPER.readTree(unwrapWithArraySchema2).get("schema"), null, this.defaultConfiguration); + Schema schema = e.icebergSchema(defaultPartitionColumn); System.out.println(schema.asStruct()); System.out.println(schema); System.out.println(schema.findField("tableChanges")); @@ -108,10 +108,9 @@ public void testNestedArray2JsonRecord() throws JsonProcessingException { public void testNestedGeomJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(unwrapWithGeomSchema).get("payload"), null, - MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, - this.defaultConfiguration); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema, "__source_ts_ms"); + MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, this.defaultConfiguration); + Schema schema = e.icebergSchema(defaultPartitionColumn); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); GenericRecord g = (GenericRecord) record.getField("g"); GenericRecord h = (GenericRecord) record.getField("h"); @@ -122,13 +121,26 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { } @Test - public void testCustomPartitionColumnRecord() throws IOException { + public void testConvertPartitionTimestampRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(customPartitionColumn).get("payload"), null, MAPPER.readTree(customPartitionColumn).get("schema"), null); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema, "timestamp"); - assertEquals("2023-03-20T18:25:27.865Z", record.getField("__source_ts").toString()); + Schema schema = e.icebergSchema(defaultPartitionColumn); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, "timestamp"); + assertEquals("2023-03-20T18:25:27.865Z", record.getField(defaultPartitionColumn).toString()); + assertEquals("hello", record.getField("message")); + System.out.println(schema); + System.out.println(record); + } + + @Test + public void testConvertPartitionColumnRecord() throws IOException { + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(customPartitionColumn).get("payload"), null, + MAPPER.readTree(customPartitionColumn).get("schema"), null); + Schema schema = e.icebergSchema("timestamp"); + GenericRecord record = e.asIcebergRecord(schema, "timestamp", "timestamp"); + assertEquals("2023-03-20T18:25:27.865Z", record.getField("timestamp").toString()); assertEquals("hello", record.getField("message")); System.out.println(schema); System.out.println(record); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperatorTest.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperatorTest.java index 58279d4..4090568 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperatorTest.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/tableoperator/IcebergTableOperatorTest.java @@ -50,7 +50,7 @@ public Table createTable(IcebergChangeEvent sampleEvent) { .build(); final TableIdentifier tableId = TableIdentifier.of(Namespace.of(TABLE_NAMESPACE), TABLE_PREFIX + sampleEvent.destinationTable()); - return IcebergUtil.createIcebergTable(icebergCatalog, tableId, sampleEvent.icebergSchema(), config); + return IcebergUtil.createIcebergTable(icebergCatalog, tableId, sampleEvent.icebergSchema(config.getPartitionColumn()), config); } @Test From 1dd5334271d6f95af5e155ff81c323511cec8da5 Mon Sep 17 00:00:00 2001 From: github-actions Date: Fri, 24 Mar 2023 09:43:34 +0000 Subject: [PATCH 30/50] FIX #31 - Bump version and CHANGELOG for release 0.3.0 --- CHANGELOG.md | 9 +++++++-- pom.xml | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 266125d..c6886c2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,7 +7,10 @@ - Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` -- Add `iceberg.partition` config setting to allow any column to be used for partitioning. + +## [0.3.0] - 2023-03-24 + +- Add `iceberg.partition` config setting to allow any column to be used for partitioning. ## [0.2.5] - 2023-03-20 @@ -50,7 +53,9 @@ Kafka Connect >= 3.2.3 has updated the jackson version to an incompatible minor - First release -[Unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.5...HEAD +[Unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.0...HEAD + +[0.3.0]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.5...0.3.0 [0.2.5]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.4...0.2.5 diff --git a/pom.xml b/pom.xml index f253df1..5289e65 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.getindata kafka-connect-iceberg-sink - 0.2.5-SNAPSHOT + 0.3.0 jar From 6b5c2bb6bb5e696da111c84ae931907473d73851 Mon Sep 17 00:00:00 2001 From: github-actions Date: Fri, 24 Mar 2023 09:50:59 +0000 Subject: [PATCH 31/50] FIX - Change version from 0.3.0 to 0.3.0-SNAPSHOT --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5289e65..511d1da 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.getindata kafka-connect-iceberg-sink - 0.3.0 + 0.3.0-SNAPSHOT jar From 656b97594b8a5db2df6e6a5037cafa69fa23d7b2 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 01:37:27 +0200 Subject: [PATCH 32/50] support for Apache Iceberg Format Version --- .../iceberg/sink/IcebergSinkConfiguration.java | 11 ++++++++++- .../kafka/connect/iceberg/sink/IcebergUtil.java | 6 +++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index fe50fcf..d58c332 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -29,6 +29,8 @@ public class IcebergSinkConfiguration { public static final String CATALOG_TYPE = ICEBERG_PREFIX + "type"; public static final String PARTITION_TIMESTAMP = ICEBERG_PREFIX + "partition.timestamp"; public static final String PARTITION_COLUMN = ICEBERG_PREFIX + "partition.column"; + public static final String FORMAT_VERSION = ICEBERG_PREFIX + "format-version"; + private static final ConfigDef CONFIG_DEF = new ConfigDef() .define(UPSERT, BOOLEAN, true, MEDIUM, @@ -68,7 +70,10 @@ public class IcebergSinkConfiguration { "will be converted to a timestamp value and stored in iceberg.partition.column") .define(PARTITION_COLUMN, STRING, "__source_ts", MEDIUM, "Column used for partitioning. If the column already exists, it must be of type timestamp.") - ; + .define(FORMAT_VERSION, STRING, "__source_ts", MEDIUM, + "Specification for the Iceberg table formatg. Version 1: Analytic Data Tables."+ + "Version 2: Row-level Deletes. Default 2."); + private final AbstractConfig parsedConfig; private final Map properties; @@ -129,6 +134,10 @@ public String getPartitionColumn() { return parsedConfig.getString(PARTITION_COLUMN); } + public String getFormatVersion() { + return parsedConfig.getString(FORMAT_VERSION); + } + /** * Gets the name of the column containing unix millisecond timestamps to be used for partitioning. * @return Unix timestamp in milliseconds diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java index 26af7c1..3c730e9 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java @@ -48,9 +48,13 @@ public static Table createIcebergTable(Catalog icebergCatalog, TableIdentifier t ps = PartitionSpec.builderFor(schema).build(); } + String formatVersion = "2"; + if(configuration.getFormatVersion()){ + formatVersion=configuration.getFormatVersion(); + } return icebergCatalog.buildTable(tableIdentifier, schema) .withProperties(configuration.getIcebergTableConfiguration()) - .withProperty(FORMAT_VERSION, "2") + .withProperty(FORMAT_VERSION, formatVersion) .withSortOrder(IcebergUtil.getIdentifierFieldsAsSortOrder(schema)) .withPartitionSpec(ps) .create(); From 999c7e46fbc16c4b8aa93eeddabf389883f13918 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 01:56:32 +0200 Subject: [PATCH 33/50] format version in README --- README.md | 84 +++++++++++++++++++++++++++++-------------------------- 1 file changed, 45 insertions(+), 39 deletions(-) diff --git a/README.md b/README.md index 17e21f4..a802ac3 100644 --- a/README.md +++ b/README.md @@ -12,44 +12,46 @@ mvn clean package ### Configuration reference -| Key | Type | Default value | Description | -|-----------------------------|---------|----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| -| upsert | boolean | true | When *true* Iceberg rows will be updated based on table primary key. When *false* all modification will be added as separate rows. | -| upsert.keep-deletes | boolean | true | When *true* delete operation will leave a tombstone that will have only a primary key and *__deleted** flag set to true | -| upsert.dedup-column | String | __source_ts_ms | Column used to check which state is newer during upsert | -| upsert.op-column | String | __op | Column used to check which state is newer during upsert when *upsert.dedup-column* is not enough to resolve | -| allow-field-addition | boolean | true | When *true* sink will be adding new columns to Iceberg tables on schema changes | -| table.auto-create | boolean | false | When *true* sink will automatically create new Iceberg tables | -| table.namespace | String | default | Table namespace. In Glue it will be used as database name | -| table.prefix | String | *empty string* | Prefix added to all table names | -| iceberg.name | String | default | Iceberg catalog name | -| iceberg.catalog-impl | String | *null* | Iceberg catalog implementation (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time | -| iceberg.type | String | *null* | Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time) | -| iceberg.* | | | All properties with this prefix will be passed to Iceberg Catalog implementation | -| iceberg.table-default.* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | -| iceberg.partition.column | String | __source_ts | Column used for partitioning. If the column already exists, it must be of type timestamp. | -| iceberg.partition.timestamp | String | __source_ts_ms | Column containing unix millisecond timestamps to be converted to partitioning times. If equal to partition.column, values will be replaced with timestamps. | +| Key | Type | Default value | Description | +| --------------------------- | ------- | ---------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------- | +| upsert | boolean | true | When _true_ Iceberg rows will be updated based on table primary key. When _false_ all modification will be added as separate rows. | +| upsert.keep-deletes | boolean | true | When _true_ delete operation will leave a tombstone that will have only a primary key and \*\_\_deleted\*\* flag set to true | +| upsert.dedup-column | String | \_\_source_ts_ms | Column used to check which state is newer during upsert | +| upsert.op-column | String | \_\_op | Column used to check which state is newer during upsert when _upsert.dedup-column_ is not enough to resolve | +| allow-field-addition | boolean | true | When _true_ sink will be adding new columns to Iceberg tables on schema changes | +| table.auto-create | boolean | false | When _true_ sink will automatically create new Iceberg tables | +| table.namespace | String | default | Table namespace. In Glue it will be used as database name | +| table.prefix | String | _empty string_ | Prefix added to all table names | +| iceberg.name | String | default | Iceberg catalog name | +| iceberg.catalog-impl | String | _null_ | Iceberg catalog implementation (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time | +| iceberg.type | String | _null_ | Iceberg catalog type (Only one of iceberg.catalog-impl and iceberg.type can be set to non null value at the same time) | +| iceberg.\* | | | All properties with this prefix will be passed to Iceberg Catalog implementation | +| iceberg.table-default.\* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | +| iceberg.partition.column | String | \_\_source_ts | Column used for partitioning. If the column already exists, it must be of type timestamp. | +| iceberg.partition.timestamp | String | \_\_source_ts_ms | Column containing unix millisecond timestamps to be converted to partitioning times. If equal to partition.column, values will be replaced with timestamps. | +| iceberg.format-version | String | \_\_source_ts_ms | Specification for the Iceberg table formatg. Version 1: Analytic Data Tables. Version 2: Row-level Deletes. Default 2. | ### REST / Manual based installation 1. Copy content of `kafka-connect-iceberg-sink-0.1.4-SNAPSHOT-plugin.zip` into Kafka Connect plugins directory. [Kafka Connect installing plugins](https://docs.confluent.io/home/connect/self-managed/userguide.html#connect-installing-plugins) 2. POST `:/connectors` + ```json { "name": "iceberg-sink", "config": { "connector.class": "com.getindata.kafka.connect.iceberg.sink.IcebergSink", "topics": "topic1,topic2", - + "upsert": true, "upsert.keep-deletes": true, - + "table.auto-create": true, "table.write-format": "parquet", "table.namespace": "my_namespace", "table.prefix": "debeziumcdc_", - + "iceberg.catalog-impl": "org.apache.iceberg.aws.glue.GlueCatalog", "iceberg.warehouse": "s3a://my_bucket/iceberg", "iceberg.fs.defaultFS": "s3a://my_bucket/iceberg", @@ -81,6 +83,7 @@ docker run -it --name connect --net=host -p 8083:8083 \ ### Strimzi KafkaConnect: + ```yaml apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaConnect @@ -122,19 +125,19 @@ spec: config.providers.secret.class: io.strimzi.kafka.KubernetesSecretConfigProvider config.providers.configmap.class: io.strimzi.kafka.KubernetesConfigMapConfigProvider build: - output: - type: docker - image: - pushSecret: - plugins: - - name: debezium-postgresql - artifacts: - - type: zip - url: https://repo1.maven.org/maven2/io/debezium/debezium-connector-postgres/2.0.0.Final/debezium-connector-postgres-2.0.0.Final-plugin.zip - - name: iceberg - artifacts: - - type: zip - url: https://github.com/TIKI-Institut/kafka-connect-iceberg-sink/releases/download/0.1.4-SNAPSHOT-hadoop-catalog-r3/kafka-connect-iceberg-sink-0.1.4-SNAPSHOT-plugin.zip + output: + type: docker + image: + pushSecret: + plugins: + - name: debezium-postgresql + artifacts: + - type: zip + url: https://repo1.maven.org/maven2/io/debezium/debezium-connector-postgres/2.0.0.Final/debezium-connector-postgres-2.0.0.Final-plugin.zip + - name: iceberg + artifacts: + - type: zip + url: https://github.com/TIKI-Institut/kafka-connect-iceberg-sink/releases/download/0.1.4-SNAPSHOT-hadoop-catalog-r3/kafka-connect-iceberg-sink-0.1.4-SNAPSHOT-plugin.zip resources: requests: cpu: "0.1" @@ -151,6 +154,7 @@ spec: ``` KafkaConnector Debezium Source + ```yaml apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaConnector @@ -182,6 +186,7 @@ spec: ``` KafkaConnector Iceberg Sink: + ```yaml apiVersion: kafka.strimzi.io/v1beta2 kind: KafkaConnector @@ -212,8 +217,8 @@ spec: iceberg.io-impl: "org.apache.iceberg.aws.s3.S3FileIO" iceberg.s3.endpoint: "http://minio:9000" iceberg.s3.path-style-access: true - iceberg.s3.access-key-id: '' - iceberg.s3.secret-access-key: '' + iceberg.s3.access-key-id: "" + iceberg.s3.secret-access-key: "" # Batch size tuning # See: https://stackoverflow.com/questions/51753883/increase-the-number-of-messages-read-by-a-kafka-consumer-in-a-single-poll # And the key prefix in Note: https://stackoverflow.com/a/66551961/2688589 @@ -225,6 +230,7 @@ spec: #### Hadoop s3a AWS credentials can be passed: + 1. As part of sink configuration under keys `iceberg.fs.s3a.access.key` and `iceberg.fs.s3a.secret.key` 2. Using enviornment variables `AWS_ACCESS_KEY` and `AWS_SECRET_ACCESS_KEY` 3. As ~/.aws/config file @@ -299,13 +305,13 @@ Similar problem is with changing optionality of a column. If it was not defined ### DML -Rows cannot be updated nor removed unless primary key is defined. In case of deletion sink behavior is also dependent on upsert.keep-deletes option. When this option is set to true sink will leave a tombstone behind in a form of row containing only a primary key value and __deleted flat set to true. When option is set to false it will remove row entirely. +Rows cannot be updated nor removed unless primary key is defined. In case of deletion sink behavior is also dependent on upsert.keep-deletes option. When this option is set to true sink will leave a tombstone behind in a form of row containing only a primary key value and \_\_deleted flat set to true. When option is set to false it will remove row entirely. ### Iceberg partitioning support The consumer reads unix millisecond timestamps from the event field configured in `iceberg.partition.timestamp`, converts them to iceberg -timestamps, and writes them to the table column configured in `iceberg.partition.column`. The timestamp column is then used to extract a -date to be used as the partitioning key. If `iceberg.partition.timestamp` is empty, `iceberg.parition.column` is assumed to already be of +timestamps, and writes them to the table column configured in `iceberg.partition.column`. The timestamp column is then used to extract a +date to be used as the partitioning key. If `iceberg.partition.timestamp` is empty, `iceberg.parition.column` is assumed to already be of type timestamp, and no conversion is performed. If they are set to the same value, the integer values will be replaced by the converted timestamp values. @@ -322,7 +328,7 @@ By default, the sink expects to receive events produced by a debezium source con ## Debezium change event format support -Kafka Connect Iceberg Sink is expecting events in a format of *Debezium change event*. It uses however only an *after* portion of that event and some metadata. +Kafka Connect Iceberg Sink is expecting events in a format of _Debezium change event_. It uses however only an _after_ portion of that event and some metadata. Minimal fields needed for the sink to work are: Kafka event key: From 03e26d59f2c190efdf98400296e895d6b4203050 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 01:56:50 +0200 Subject: [PATCH 34/50] add test for format version --- .../connect/iceberg/sink/TestIcebergUtil.java | 24 +++++++++++++++++++ .../sink/testresources/TestConfig.java | 5 ++++ 2 files changed, 29 insertions(+) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index bea72b2..8ebc0b3 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -325,6 +325,30 @@ public void createIcebergTablesWithCustomProperties(@TempDir Path localWarehouse assertTrue(IcebergUtil.getTableFileFormat(table1) == FileFormat.ORC); } + @Test + public void createIcebergTablesWithCustomPropertiesFormatVersion(@TempDir Path localWarehouseDir) { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withUpsert(false) + .withCustomCatalogProperty("table-default.write.format.default", "orc") + .withFormatVersion("1") + .build(); + + Catalog catalog = IcebergCatalogFactory.create(config); + + Schema schema = new Schema( + List.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())), + Set.of(1) + ); + + Table table1 = IcebergUtil.createIcebergTable(catalog, TableIdentifier.of("test", "test"), schema, config); + + assertTrue(IcebergUtil.getTableFileFormat(table1) == FileFormat.ORC); + } + + @Test public void testToSnakeCase() { assertTrue(IcebergUtil.toSnakeCase("armadillo_pension").equals("armadillo_pension")); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java index 4f0883e..2300d82 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java @@ -69,6 +69,11 @@ public Builder withCustomProperty(String key, String value) { return this; } + public Builder withFormatVersion(String formatVersion) { + properties.put(IcebergSinkConfiguration.FORMAT_VERSION, formatVersion)); + return this; + } + public IcebergSinkConfiguration build() { return new IcebergSinkConfiguration(properties); } From 8d6f8b1f7396334783bfd197530ae84f263dd3a0 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 02:04:36 +0200 Subject: [PATCH 35/50] improv bad test --- .../com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java index 3c730e9..fd471f3 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java @@ -49,7 +49,7 @@ public static Table createIcebergTable(Catalog icebergCatalog, TableIdentifier t } String formatVersion = "2"; - if(configuration.getFormatVersion()){ + if(configuration.getFormatVersion() != null && !"".equals(configuration.getFormatVersion())){ formatVersion=configuration.getFormatVersion(); } return icebergCatalog.buildTable(tableIdentifier, schema) From f117fa6d5ae305f748d4ee8c3cf985236c64be1b Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 02:11:41 +0200 Subject: [PATCH 36/50] compilation failure bad syntax --- .../kafka/connect/iceberg/sink/testresources/TestConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java index 2300d82..7b67f6d 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java @@ -70,7 +70,7 @@ public Builder withCustomProperty(String key, String value) { } public Builder withFormatVersion(String formatVersion) { - properties.put(IcebergSinkConfiguration.FORMAT_VERSION, formatVersion)); + properties.put(IcebergSinkConfiguration.FORMAT_VERSION, formatVersion); return this; } From 3999bb49eee6f175126597646865db084739c485 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 29 Mar 2023 02:18:27 +0200 Subject: [PATCH 37/50] change default value --- .../kafka/connect/iceberg/sink/IcebergSinkConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index d58c332..9a17820 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -70,7 +70,7 @@ public class IcebergSinkConfiguration { "will be converted to a timestamp value and stored in iceberg.partition.column") .define(PARTITION_COLUMN, STRING, "__source_ts", MEDIUM, "Column used for partitioning. If the column already exists, it must be of type timestamp.") - .define(FORMAT_VERSION, STRING, "__source_ts", MEDIUM, + .define(FORMAT_VERSION, STRING, "2", MEDIUM, "Specification for the Iceberg table formatg. Version 1: Analytic Data Tables."+ "Version 2: Row-level Deletes. Default 2."); From 1d2f5e928a63f4cd1e83082239bd83a664713aa7 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 5 Apr 2023 11:19:13 +0200 Subject: [PATCH 38/50] PR #32 Third column is a default value. According to code its "2" --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a802ac3..a0e3b3f 100644 --- a/README.md +++ b/README.md @@ -29,7 +29,7 @@ mvn clean package | iceberg.table-default.\* | | | Iceberg specific table settings can be changed with this prefix, e.g. 'iceberg.table-default.write.format.default' can be set to 'orc' | | iceberg.partition.column | String | \_\_source_ts | Column used for partitioning. If the column already exists, it must be of type timestamp. | | iceberg.partition.timestamp | String | \_\_source_ts_ms | Column containing unix millisecond timestamps to be converted to partitioning times. If equal to partition.column, values will be replaced with timestamps. | -| iceberg.format-version | String | \_\_source_ts_ms | Specification for the Iceberg table formatg. Version 1: Analytic Data Tables. Version 2: Row-level Deletes. Default 2. | +| iceberg.format-version | String | 2 | Specification for the Iceberg table format. Version 1: Analytic Data Tables. Version 2: Row-level Deletes. Default 2. | ### REST / Manual based installation From 90f264f1825d1d787b59e02f13751506d54d7a27 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 5 Apr 2023 11:21:41 +0200 Subject: [PATCH 39/50] PR #32 formatting: there should be spaces around equal sign --- .../connect/iceberg/sink/IcebergUtil.java | 68 +++++++++---------- 1 file changed, 34 insertions(+), 34 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java index fd471f3..21aff48 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergUtil.java @@ -35,7 +35,7 @@ public class IcebergUtil { protected static final ObjectMapper jsonObjectMapper = new ObjectMapper(); public static Table createIcebergTable(Catalog icebergCatalog, TableIdentifier tableIdentifier, - Schema schema, IcebergSinkConfiguration configuration) { + Schema schema, IcebergSinkConfiguration configuration) { LOGGER.info("Creating table:'{}'\nschema:{}\nrowIdentifier:{}", tableIdentifier, schema, schema.identifierFieldNames()); @@ -49,8 +49,8 @@ public static Table createIcebergTable(Catalog icebergCatalog, TableIdentifier t } String formatVersion = "2"; - if(configuration.getFormatVersion() != null && !"".equals(configuration.getFormatVersion())){ - formatVersion=configuration.getFormatVersion(); + if (configuration.getFormatVersion() != null && !"".equals(configuration.getFormatVersion())) { + formatVersion = configuration.getFormatVersion(); } return icebergCatalog.buildTable(tableIdentifier, schema) .withProperties(configuration.getIcebergTableConfiguration()) @@ -94,43 +94,43 @@ public static GenericAppenderFactory getTableAppender(Table icebergTable) { } public static String toSnakeCase(String inputString) { - - StringBuilder sb = new StringBuilder(); - boolean lastUpper = true; - boolean lastSeparator = false; - for (Character c : inputString.toCharArray()) { - - if (Character.isUpperCase(c)) { + StringBuilder sb = new StringBuilder(); + boolean lastUpper = true; + boolean lastSeparator = false; - if (!lastUpper) { + for (Character c : inputString.toCharArray()) { - if (!lastSeparator) { - sb.append("_"); - } + if (Character.isUpperCase(c)) { - lastUpper = true; - } + if (!lastUpper) { - sb.append(Character.toLowerCase(c)); - lastSeparator = false; - } - - else { - - if (c == '_') { - lastSeparator = true; - } - - else { - lastSeparator = false; - } - - sb.append(c); - lastUpper = false; + if (!lastSeparator) { + sb.append("_"); } + + lastUpper = true; + } + + sb.append(Character.toLowerCase(c)); + lastSeparator = false; + } + + else { + + if (c == '_') { + lastSeparator = true; + } + + else { + lastSeparator = false; + } + + sb.append(c); + lastUpper = false; } - - return sb.toString(); + } + + return sb.toString(); } } From 8298f0b9be512ef370d48fd077913116df5fa2f9 Mon Sep 17 00:00:00 2001 From: Charly CLAIRMONT Date: Wed, 5 Apr 2023 22:11:32 +0200 Subject: [PATCH 40/50] indicate iceberg.format-version in the changelog --- CHANGELOG.md | 42 +++++++++++++++++------------------------- 1 file changed, 17 insertions(+), 25 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c6886c2..6c2aeb1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,68 +7,60 @@ - Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` +- Add `iceberg.format-version` config setting to indicate which Iceberg table format version is used. ## [0.3.0] - 2023-03-24 -- Add `iceberg.partition` config setting to allow any column to be used for partitioning. +- Add `iceberg.partition` config setting to allow any column to be used for partitioning. ## [0.2.5] - 2023-03-20 -- Reverted pom.xml groupid +- Reverted pom.xml groupid ## [0.2.4] - 2023-03-13 -- Added support for `double` primitive type fields. -- Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. +- Added support for `double` primitive type fields. +- Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. ## [0.2.2] - 2023-02-17 -- Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties +- Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties ## [0.2.1] - 2022-12-09 -- removed 'table.write-format', can be replaced with 'iceberg.table-default.write.format.default' +- removed 'table.write-format', can be replaced with 'iceberg.table-default.write.format.default' ## [0.2.0] - 2022-11-16 -- Added support for Hive metastore catalog -- Replaced maven-shade plugin with maven-assembly. To add hadoop default configuration files -- Integrated updates from -- Updated Iceberg to 1.0.0 -- Updated to Kafka Connect API 3.2.2 +- Added support for Hive metastore catalog +- Replaced maven-shade plugin with maven-assembly. To add hadoop default configuration files +- Integrated updates from +- Updated Iceberg to 1.0.0 +- Updated to Kafka Connect API 3.2.2 ### Version Compatibility -This Iceberg Sink depends on a Spark 3.2 Runtime, which depends on a specific jackson minor version. +This Iceberg Sink depends on a Spark 3.2 Runtime, which depends on a specific jackson minor version. Kafka Connect >= 3.2.3 has updated the jackson version to an incompatible minor release (2.13) ## [0.1.3] - 2022-04-11 -- Logger levels changes -- Added documentation to sink configuration +- Logger levels changes +- Added documentation to sink configuration ## [0.1.2] - 2022-03-25 ## [0.1.1] - 2022-03-25 -- First release - -[Unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.0...HEAD +- First release +[unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.0...HEAD [0.3.0]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.5...0.3.0 - [0.2.5]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.4...0.2.5 - [0.2.4]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.2...0.2.4 - [0.2.2]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.1...0.2.2 - [0.2.1]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.0...0.2.1 - [0.2.0]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.3...0.2.0 - [0.1.3]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.2...0.1.3 - [0.1.2]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.1...0.1.2 - [0.1.1]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/1190003ddc686273cb9ad28ce7dd2d8e458471d7...0.1.1 From 6d1a516ae17406f63245b9cc9ddd2a7b1413f16c Mon Sep 17 00:00:00 2001 From: github-actions Date: Thu, 6 Apr 2023 05:44:26 +0000 Subject: [PATCH 41/50] FIX #34 - Bump version and CHANGELOG for release 0.3.1 --- CHANGELOG.md | 46 ++++++++++++++++++++++++++++++---------------- pom.xml | 2 +- 2 files changed, 31 insertions(+), 17 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6c2aeb1..29de772 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,36 +7,39 @@ - Added Iceberg coercion support for Avro Array types. Supports Debezium `data_collections` metadata. - Added support for coercion of five Debezium temporal types to their Iceberg equivalents: Date, MicroTimestamp, ZonedTimestamp, MicroTime, and ZonedTime - Rich temporal types are toggled on by new boolean configuration property: `rich-temporal-types` -- Add `iceberg.format-version` config setting to indicate which Iceberg table format version is used. + +## [0.3.1] - 2023-04-06 + +- Add `iceberg.format-version` config setting to indicate which Iceberg table format version is used. ## [0.3.0] - 2023-03-24 -- Add `iceberg.partition` config setting to allow any column to be used for partitioning. +- Add `iceberg.partition` config setting to allow any column to be used for partitioning. ## [0.2.5] - 2023-03-20 -- Reverted pom.xml groupid +- Reverted pom.xml groupid ## [0.2.4] - 2023-03-13 -- Added support for `double` primitive type fields. -- Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. +- Added support for `double` primitive type fields. +- Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. ## [0.2.2] - 2023-02-17 -- Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties +- Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties ## [0.2.1] - 2022-12-09 -- removed 'table.write-format', can be replaced with 'iceberg.table-default.write.format.default' +- removed 'table.write-format', can be replaced with 'iceberg.table-default.write.format.default' ## [0.2.0] - 2022-11-16 -- Added support for Hive metastore catalog -- Replaced maven-shade plugin with maven-assembly. To add hadoop default configuration files -- Integrated updates from -- Updated Iceberg to 1.0.0 -- Updated to Kafka Connect API 3.2.2 +- Added support for Hive metastore catalog +- Replaced maven-shade plugin with maven-assembly. To add hadoop default configuration files +- Integrated updates from +- Updated Iceberg to 1.0.0 +- Updated to Kafka Connect API 3.2.2 ### Version Compatibility @@ -45,22 +48,33 @@ Kafka Connect >= 3.2.3 has updated the jackson version to an incompatible minor ## [0.1.3] - 2022-04-11 -- Logger levels changes -- Added documentation to sink configuration +- Logger levels changes +- Added documentation to sink configuration ## [0.1.2] - 2022-03-25 ## [0.1.1] - 2022-03-25 -- First release +- First release + +[Unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.1...HEAD + +[0.3.1]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.0...0.3.1 -[unreleased]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.3.0...HEAD [0.3.0]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.5...0.3.0 + [0.2.5]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.4...0.2.5 + [0.2.4]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.2...0.2.4 + [0.2.2]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.1...0.2.2 + [0.2.1]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.2.0...0.2.1 + [0.2.0]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.3...0.2.0 + [0.1.3]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.2...0.1.3 + [0.1.2]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/0.1.1...0.1.2 + [0.1.1]: https://github.com/getindata/kafka-connect-iceberg-sink/compare/1190003ddc686273cb9ad28ce7dd2d8e458471d7...0.1.1 diff --git a/pom.xml b/pom.xml index 511d1da..7e929bb 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ com.getindata kafka-connect-iceberg-sink - 0.3.0-SNAPSHOT + 0.3.1 jar From 953dc9ff0d0926aac714db9c9e5f050278633cdb Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 16 Mar 2023 10:33:28 -0700 Subject: [PATCH 42/50] initial commit message for added debezium date and micro timestamp coercion support --- .../iceberg/sink/IcebergChangeEvent.java | 17 ++++++++++++++--- .../iceberg/sink/IcebergSinkConfiguration.java | 7 +++++++ .../connect/iceberg/sink/IcebergSinkTask.java | 2 ++ 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index cf52ac5..8975192 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -163,7 +163,6 @@ private Object jsonValToIcebergVal(Types.NestedField field, String fieldTypeName = field.doc(); LOGGER.debug("Processing Field:{} Type:{} Doc:{}", field.name(), field.type(), fieldTypeName); - final Object val; switch (field.type().typeId()) { case INTEGER: // int 4 bytes @@ -217,8 +216,20 @@ else if (node.isNull()){ } break; case STRING: - // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); + // string destination coercions based upon schema 'name' annotations + if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { + val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); + } + else { + if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && + fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { + val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); + } + else { + // if the node is not a value node (method isValueNode returns false), convert it to string. + val = node.isValueNode() ? node.asText(null) : node.toString(); + } + } break; case BINARY: try { diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index 9a17820..821a5b6 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -11,6 +11,8 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; +import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; + public class IcebergSinkConfiguration { public static final String UPSERT = "upsert"; public static final String UPSERT_KEEP_DELETES = "upsert.keep-deletes"; @@ -169,4 +171,9 @@ public static ConfigDef getConfigDef() { public Map getProperties() { return properties; } + + public void configureChangeEvent() { + IcebergChangeEvent.setCoerceDebeziumDate(this.isCoerceDebeziumDate()); + IcebergChangeEvent.setCoerceDebeziumMicroTimestamp(this.isCoerceDebeziumMicroTimestamp()); + } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java index a41cb4f..3fea10a 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java @@ -26,6 +26,8 @@ public String version() { public void start(Map properties) { LOGGER.info("Task starting"); IcebergSinkConfiguration configuration = new IcebergSinkConfiguration(properties); + // provide type coercion configuration IcebergChangeEvent + configuration.configureChangeEvent(); Catalog icebergCatalog = IcebergCatalogFactory.create(configuration); IcebergTableOperator icebergTableOperator = IcebergTableOperatorFactory.create(configuration); SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(configuration); From 06e2dc207a9c7265f02dca4043d68e1ec5105e2c Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:21:00 -0700 Subject: [PATCH 43/50] repair unit tests after adding type annotation information to schema --- .../kafka/connect/iceberg/sink/TestIcebergUtil.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index 8ebc0b3..b542c65 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -59,8 +59,8 @@ public void testNestedJsonRecord() throws JsonProcessingException { MAPPER.readTree(serdeWithSchema).get("payload"), null, MAPPER.readTree(serdeWithSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); - assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), 3: first_name: optional string (), " + - "4:")); + assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), " + + "3: first_name: optional string (), 4:")); } @Test @@ -81,7 +81,6 @@ public void testNestedArrayJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); - System.out.println(schema.asStruct()); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); From 7b41f165474a2d1bc39c1145b8590f2112b50519 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:22:30 -0700 Subject: [PATCH 44/50] Added unit tests for Debezium time coercions. Unit tests demonstrate that coercions should not applied by default, without configuration changes, should not be applied when explicitly disabled via configuration, and are only applied when explicitly configured via IcebergSinkConfiguration. Both values and types are tested. --- .../connect/iceberg/sink/TestIcebergUtil.java | 67 +++++++++++++++++++ .../sink/testresources/TestConfig.java | 5 ++ 2 files changed, 72 insertions(+) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index b542c65..c6c2e90 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -302,6 +302,73 @@ public void listStructSchemaHandling() assertEquals(1680821545908L, value); } + @Test + public void coerceDebeziumTimeTypesDefaultBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesDisabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "false") + .withCustomProperty("coerce.debezium-micro-timestamp", "false") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("77663")); + assertTrue(recordString.contains("6710075456016196")); + } + + @Test + public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseDir) + throws JsonProcessingException { + IcebergSinkConfiguration config = TestConfig.builder() + .withLocalCatalog(localWarehouseDir) + .withCustomProperty("coerce.debezium-date", "true") + .withCustomProperty("coerce.debezium-micro-timestamp", "true") + .build(); + config.configureChangeEvent(); + IcebergChangeEvent e = new IcebergChangeEvent("test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + Schema schema = e.icebergSchema(); + GenericRecord record = e.asIcebergRecord(schema); + String schemaString = schema.toString(); + String recordString = record.toString(); + + assertTrue(schemaString.contains("ship_date: optional string (io.debezium.time.Date)")); + assertTrue(schemaString.contains("ship_timestamp: optional string (io.debezium.time.MicroTimestamp)")); + assertTrue(recordString.contains("2182-08-20")); + assertTrue(recordString.contains("2182-08-19T21:50:56.016196Z")); + } + @Test public void createIcebergTablesWithCustomProperties(@TempDir Path localWarehouseDir) { IcebergSinkConfiguration config = TestConfig.builder() diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java index 7b67f6d..2dc7c52 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/testresources/TestConfig.java @@ -74,6 +74,11 @@ public Builder withFormatVersion(String formatVersion) { return this; } + public Builder withCustomProperty(String key, String value) { + properties.put(key, value); + return this; + } + public IcebergSinkConfiguration build() { return new IcebergSinkConfiguration(properties); } From 99b8969e3f95a39c4506b8db91ce142a9d1e4080 Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:34:41 -0700 Subject: [PATCH 45/50] updated CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 29de772..b47e3e4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,7 +24,6 @@ - Added support for `double` primitive type fields. - Allow coercion of iceberg table identifiers to `snake_case` setting `table.snake-case` boolean configuration. - ## [0.2.2] - 2023-02-17 - Allow changing iceberg-table specific settings using `iceberg.table-default.*` connector configuration properties From 1df95ada562e1beeb681faedf7bbcc2867d3a10c Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 17 Mar 2023 08:21:00 -0700 Subject: [PATCH 46/50] repair unit tests after adding type annotation information to schema --- .../getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index c6c2e90..b5f874d 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -80,7 +80,8 @@ public void testNestedArrayJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithArraySchema).get("payload"), null, MAPPER.readTree(unwrapWithArraySchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); - assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), 2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); + assertTrue(schema.asStruct().toString().contains("struct<1: name: optional string (), " + + "2: pay_by_quarter: optional list (), 4: schedule: optional list (), 6:")); System.out.println(schema.findField("pay_by_quarter").type().asListType().elementType()); System.out.println(schema.findField("schedule").type().asListType().elementType()); assertEquals(schema.findField("pay_by_quarter").type().asListType().elementType().toString(), "int"); From c733572536173a721c13ad6f682d34413517b728 Mon Sep 17 00:00:00 2001 From: dacreify Date: Wed, 12 Apr 2023 13:29:53 -0700 Subject: [PATCH 47/50] add rich temporal support for zoned timestamps and time values, plumb through config object to events --- .../sink/IcebergSinkConfiguration.java | 7 -- .../connect/iceberg/sink/IcebergSinkTask.java | 2 - .../connect/iceberg/sink/TestIcebergUtil.java | 67 ++++++++----------- 3 files changed, 28 insertions(+), 48 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java index 821a5b6..9a17820 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkConfiguration.java @@ -11,8 +11,6 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; -import com.getindata.kafka.connect.iceberg.sink.IcebergChangeEvent; - public class IcebergSinkConfiguration { public static final String UPSERT = "upsert"; public static final String UPSERT_KEEP_DELETES = "upsert.keep-deletes"; @@ -171,9 +169,4 @@ public static ConfigDef getConfigDef() { public Map getProperties() { return properties; } - - public void configureChangeEvent() { - IcebergChangeEvent.setCoerceDebeziumDate(this.isCoerceDebeziumDate()); - IcebergChangeEvent.setCoerceDebeziumMicroTimestamp(this.isCoerceDebeziumMicroTimestamp()); - } } diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java index 3fea10a..a41cb4f 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergSinkTask.java @@ -26,8 +26,6 @@ public String version() { public void start(Map properties) { LOGGER.info("Task starting"); IcebergSinkConfiguration configuration = new IcebergSinkConfiguration(properties); - // provide type coercion configuration IcebergChangeEvent - configuration.configureChangeEvent(); Catalog icebergCatalog = IcebergCatalogFactory.create(configuration); IcebergTableOperator icebergTableOperator = IcebergTableOperatorFactory.create(configuration); SinkRecordToIcebergChangeEventConverter converter = SinkRecordToIcebergChangeEventConverterFactory.create(configuration); diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index b5f874d..b67cdbd 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -53,6 +53,8 @@ class TestIcebergUtil { final String customPartitionColumn = Testing.Files.readResourceAsString("json/custom-partition-column.json"); + private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); + @Test public void testNestedJsonRecord() throws JsonProcessingException { IcebergChangeEvent e = new IcebergChangeEvent("test", @@ -304,61 +306,48 @@ public void listStructSchemaHandling() } @Test - public void coerceDebeziumTimeTypesDefaultBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesDefaultBehavior() throws JsonProcessingException { - IcebergSinkConfiguration config = TestConfig.builder() - .withLocalCatalog(localWarehouseDir) - .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); - String schemaString = schema.toString(); - String recordString = record.toString(); + IcebergChangeEvent event = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + this.defaultConfiguration + ); - assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); - assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); - assertTrue(recordString.contains("77663")); - assertTrue(recordString.contains("6710075456016196")); + assertPrimitiveTemporalValues(event); } @Test - public void coerceDebeziumTimeTypesDisabledBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesDisabledBehavior(@TempDir Path localWarehouseDir) throws JsonProcessingException { IcebergSinkConfiguration config = TestConfig.builder() .withLocalCatalog(localWarehouseDir) - .withCustomProperty("coerce.debezium-date", "false") - .withCustomProperty("coerce.debezium-micro-timestamp", "false") + .withCustomProperty("rich-temporal-types", "false") .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); - Schema schema = e.icebergSchema(); - GenericRecord record = e.asIcebergRecord(schema); - String schemaString = schema.toString(); - String recordString = record.toString(); + IcebergChangeEvent event = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + config + ); - assertTrue(schemaString.contains("ship_date: optional int (io.debezium.time.Date)")); - assertTrue(schemaString.contains("ship_timestamp: optional long (io.debezium.time.MicroTimestamp)")); - assertTrue(recordString.contains("77663")); - assertTrue(recordString.contains("6710075456016196")); + assertPrimitiveTemporalValues(event); } @Test - public void coerceDebeziumTimeTypesEnabledBehavior(@TempDir Path localWarehouseDir) + public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWarehouseDir) throws JsonProcessingException { - IcebergSinkConfiguration config = TestConfig.builder() + IcebergSinkConfiguration configuration = TestConfig.builder() .withLocalCatalog(localWarehouseDir) - .withCustomProperty("coerce.debezium-date", "true") - .withCustomProperty("coerce.debezium-micro-timestamp", "true") + .withCustomProperty("rich-temporal-types", "true") .build(); - config.configureChangeEvent(); - IcebergChangeEvent e = new IcebergChangeEvent("test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null); + IcebergChangeEvent e = new IcebergChangeEvent( + "test", + MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, + MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, + configuration + ); Schema schema = e.icebergSchema(); GenericRecord record = e.asIcebergRecord(schema); String schemaString = schema.toString(); From 36472b886b03e1f1d30b39a5692a3d98d7e2729f Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Fri, 7 Apr 2023 14:10:41 -0700 Subject: [PATCH 48/50] Added support for Array types containing nested Struct element types such as "io.debezium.connector.common.TransactionMetadataValue" Added to support metadata via debezium connector config: "provide.transaction.metadata": "true" --- .../getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index b67cdbd..afc8c6f 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -52,6 +52,7 @@ class TestIcebergUtil { private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); final String customPartitionColumn = Testing.Files.readResourceAsString("json/custom-partition-column.json"); + final String debeziumMetadataSchema = Testing.Files.readResourceAsString("json/debezium-metadata-schema.json"); private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); From f47848806cda674cc96291b9f2b02498a92af3ae Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Thu, 13 Apr 2023 14:29:22 -0700 Subject: [PATCH 49/50] updated CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b47e3e4..00780e8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,7 @@ # Changelog +- Updates AWS lakeformation transitive dependency providing lakeformation support in s3 iceberg tables. + ## [Unreleased] - Updates dependencies to resolve some jackson-databind critical CVEs. From e5d2479e779b9fb66967d6dafff48f2bfca0d3ea Mon Sep 17 00:00:00 2001 From: Christopher Penrose Date: Mon, 1 May 2023 10:51:06 -0700 Subject: [PATCH 50/50] Repair unit tests post rebase Repair merge duplications. --- .../iceberg/sink/IcebergChangeEvent.java | 15 +-- .../connect/iceberg/sink/TestIcebergUtil.java | 102 +++--------------- .../sink/testresources/TestConfig.java | 5 - .../json/debezium-annotated-schema.json | 1 + 4 files changed, 15 insertions(+), 108 deletions(-) diff --git a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java index 8975192..82d0690 100644 --- a/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java +++ b/src/main/java/com/getindata/kafka/connect/iceberg/sink/IcebergChangeEvent.java @@ -216,20 +216,7 @@ else if (node.isNull()){ } break; case STRING: - // string destination coercions based upon schema 'name' annotations - if (IcebergChangeEvent.coerceDebeziumDate && fieldTypeName.equals("io.debezium.time.Date")) { - val = node.isNull() ? null : LocalDate.ofEpochDay(node.asInt()).toString(); - } - else { - if (IcebergChangeEvent.coerceDebeziumMicroTimestamp && - fieldTypeName.equals("io.debezium.time.MicroTimestamp")) { - val = node.isNull() ? null : Instant.ofEpochSecond(0L, node.asLong() * 1000).toString(); - } - else { - // if the node is not a value node (method isValueNode returns false), convert it to string. - val = node.isValueNode() ? node.asText(null) : node.toString(); - } - } + val = node.isValueNode() ? node.asText(null) : node.toString(); break; case BINARY: try { diff --git a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java index afc8c6f..42e64f9 100644 --- a/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java +++ b/src/test/java/com/getindata/kafka/connect/iceberg/sink/TestIcebergUtil.java @@ -48,11 +48,7 @@ class TestIcebergUtil { final String unwrapWithArraySchema2 = Testing.Files.readResourceAsString("json/serde-with-array2.json"); final String debeziumTimeCoercionSchema = Testing.Files.readResourceAsString("json/debezium-annotated-schema.json"); final String debeziumMetadataSchema = Testing.Files.readResourceAsString("json/debezium-metadata-schema.json"); - - private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); - final String customPartitionColumn = Testing.Files.readResourceAsString("json/custom-partition-column.json"); - final String debeziumMetadataSchema = Testing.Files.readResourceAsString("json/debezium-metadata-schema.json"); private final IcebergSinkConfiguration defaultConfiguration = new IcebergSinkConfiguration(new HashMap()); @@ -62,7 +58,7 @@ public void testNestedJsonRecord() throws JsonProcessingException { MAPPER.readTree(serdeWithSchema).get("payload"), null, MAPPER.readTree(serdeWithSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); - assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), " + + assertTrue(schema.toString().contains("before: optional struct<2: id: optional int (), " + "3: first_name: optional string (), 4:")); } @@ -113,6 +109,7 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { MAPPER.readTree(unwrapWithGeomSchema).get("payload"), null, MAPPER.readTree(unwrapWithGeomSchema).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertTrue(schema.toString().contains("g: optional struct<3: wkb: optional string (), 4: srid: optional int ()>")); GenericRecord g = (GenericRecord) record.getField("g"); @@ -127,7 +124,7 @@ public void testNestedGeomJsonRecord() throws JsonProcessingException { public void testConvertPartitionTimestampRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(customPartitionColumn).get("payload"), null, - MAPPER.readTree(customPartitionColumn).get("schema"), null); + MAPPER.readTree(customPartitionColumn).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema(defaultPartitionColumn); GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, "timestamp"); assertEquals("2023-03-20T18:25:27.865Z", record.getField(defaultPartitionColumn).toString()); @@ -140,7 +137,7 @@ public void testConvertPartitionTimestampRecord() throws IOException { public void testConvertPartitionColumnRecord() throws IOException { IcebergChangeEvent e = new IcebergChangeEvent("test", MAPPER.readTree(customPartitionColumn).get("payload"), null, - MAPPER.readTree(customPartitionColumn).get("schema"), null); + MAPPER.readTree(customPartitionColumn).get("schema"), null, this.defaultConfiguration); Schema schema = e.icebergSchema("timestamp"); GenericRecord record = e.asIcebergRecord(schema, "timestamp", "timestamp"); assertEquals("2023-03-20T18:25:27.865Z", record.getField("timestamp").toString()); @@ -171,7 +168,7 @@ public void valuePayloadWithSchemaAsJsonNode() { } private void assertPrimitiveTemporalValues(IcebergChangeEvent event) { - Schema schema = event.icebergSchema(); + Schema schema = event.icebergSchema(defaultPartitionColumn); Types.NestedField ship_date = schema.findField("ship_date"); assertEquals(Types.IntegerType.get(), ship_date.type()); @@ -193,7 +190,7 @@ private void assertPrimitiveTemporalValues(IcebergChangeEvent event) { assertEquals(Types.StringType.get(), ship_time_zoned.type()); assertEquals("io.debezium.time.ZonedTime", ship_time_zoned.doc()); - GenericRecord record = event.asIcebergRecord(schema); + GenericRecord record = event.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertEquals(record.getField("ship_date"), 77663); assertEquals(record.getField("ship_timestamp"), 6710075456016196L); assertEquals(record.getField("ship_timestamp_zoned"), "2023-04-11T20:32:46.821144Z"); @@ -201,79 +198,6 @@ private void assertPrimitiveTemporalValues(IcebergChangeEvent event) { assertEquals(record.getField("ship_time_zoned"), "20:32:46.821144Z"); } - @Test - public void coerceDebeziumTemporalTypesDefaultBehavior() - throws JsonProcessingException { - IcebergChangeEvent event = new IcebergChangeEvent( - "test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, - this.defaultConfiguration - ); - - assertPrimitiveTemporalValues(event); - } - - @Test - public void coerceDebeziumTemporalTypesDisabledBehavior(@TempDir Path localWarehouseDir) - throws JsonProcessingException { - IcebergSinkConfiguration config = TestConfig.builder() - .withLocalCatalog(localWarehouseDir) - .withCustomProperty("rich-temporal-types", "false") - .build(); - IcebergChangeEvent event = new IcebergChangeEvent( - "test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, - config - ); - - assertPrimitiveTemporalValues(event); - } - - @Test - public void coerceDebeziumTemporalTypesEnabledBehavior(@TempDir Path localWarehouseDir) - throws JsonProcessingException { - IcebergSinkConfiguration configuration = TestConfig.builder() - .withLocalCatalog(localWarehouseDir) - .withCustomProperty("rich-temporal-types", "true") - .build(); - IcebergChangeEvent e = new IcebergChangeEvent( - "test", - MAPPER.readTree(debeziumTimeCoercionSchema).get("payload"), null, - MAPPER.readTree(debeziumTimeCoercionSchema).get("schema"), null, - configuration - ); - Schema schema = e.icebergSchema(); - - Types.NestedField ship_date = schema.findField("ship_date"); - assertEquals(ship_date.type(), Types.DateType.get()); - assertEquals(ship_date.doc(), "io.debezium.time.Date"); - - Types.NestedField ship_timestamp = schema.findField("ship_timestamp"); - assertEquals(ship_timestamp.type(), Types.TimestampType.withoutZone()); - assertEquals(ship_timestamp.doc(), "io.debezium.time.MicroTimestamp"); - - Types.NestedField ship_timestamp_zoned = schema.findField("ship_timestamp_zoned"); - assertEquals(ship_timestamp_zoned.type(), Types.TimestampType.withZone()); - assertEquals(ship_timestamp_zoned.doc(), "io.debezium.time.ZonedTimestamp"); - - Types.NestedField ship_time = schema.findField("ship_time"); - assertEquals(ship_time.type(), Types.TimeType.get()); - assertEquals(ship_time.doc(), "io.debezium.time.MicroTime"); - - Types.NestedField ship_time_zoned = schema.findField("ship_time_zoned"); - assertEquals(ship_time_zoned.type(), Types.TimeType.get()); - assertEquals(ship_time_zoned.doc(), "io.debezium.time.ZonedTime"); - - GenericRecord record = e.asIcebergRecord(schema); - assertEquals(record.getField("ship_date"), LocalDate.parse("2182-08-20")); - assertEquals(record.getField("ship_timestamp"), LocalDateTime.parse("2182-08-19T21:50:56.016196")); - assertEquals(record.getField("ship_timestamp_zoned"), OffsetDateTime.parse("2023-04-11T20:32:46.821144Z")); - assertEquals(record.getField("ship_time"), LocalTime.ofNanoOfDay(73966821144L * 1000)); - assertEquals(record.getField("ship_time_zoned"), OffsetTime.parse("20:32:46.821144Z").toLocalTime()); - } - @Test public void listStructSchemaHandling() throws JsonProcessingException { @@ -282,10 +206,10 @@ public void listStructSchemaHandling() MAPPER.readTree(debeziumMetadataSchema).get("schema"), null, defaultConfiguration ); - Schema schema = e.icebergSchema(); + Schema schema = e.icebergSchema(defaultPartitionColumn); String schemaString = schema.toString(); - GenericRecord record = e.asIcebergRecord(schema); + GenericRecord record = e.asIcebergRecord(schema, defaultPartitionColumn, defaultPartitionTimestamp); assertTrue(schemaString.contains("data_collections: optional list