From 03c3e36c6f13dc126386485fa2aebf5a5ef9bf5a Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 4 Apr 2024 09:58:18 -0600 Subject: [PATCH 01/32] dead-letter-table - implements dead letter table handling as a config option --- .../iceberg/connect/IcebergSinkConfig.java | 5 + .../iceberg/connect/channel/Worker.java | 197 ++++++++++++------ 2 files changed, 139 insertions(+), 63 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 1c212879..c278a904 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -333,6 +333,11 @@ public boolean dynamicTablesEnabled() { return getBoolean(TABLES_DYNAMIC_PROP); } + // TODO + public boolean deadLetterTableEnabled() { + return false; + } + public String tablesRouteField() { return getString(TABLES_ROUTE_FIELD_PROP); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 512471c7..056a0285 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -53,11 +53,128 @@ public class Worker extends Channel { private final IcebergSinkConfig config; - private final IcebergWriterFactory writerFactory; +// private final IcebergWriterFactory writerFactory; private final SinkTaskContext context; private final String controlGroupId; private final Map writers; private final Map sourceOffsets; + private final RecordRouter recordRouter; + + private interface WriterForTable { + void write(String tableName, SinkRecord sample, boolean ignoreMissingTable); + } + + private static class BaseWriterForTable implements WriterForTable { + + private final IcebergWriterFactory writerFactory; + private final Map writers; + + BaseWriterForTable(IcebergWriterFactory writerFactory, Map writers) { + this.writerFactory = writerFactory; + this.writers = writers; + } + + @Override + public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { + writers.computeIfAbsent( + tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)).write(sample); + + } + } + + private static class DeadLetterWriterForTable implements WriterForTable { + private final IcebergWriterFactory writerFactory; + private final Map writers; + + DeadLetterWriterForTable(IcebergWriterFactory writerFactory, Map writers) { + this.writerFactory = writerFactory; + this.writers = writers; + } + + @Override + public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { + RecordWriter writer = writers.computeIfAbsent( + tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); + try { + writer.write(sample); + } catch (Exception e) { + // dig out the bytes + // generate the table name (topic + _dlt) ? + // writers.computeIfAbsent + // write the message. + } + } + } + + private abstract class RecordRouter { + void write(SinkRecord record) {} + + protected String extractRouteValue(Object recordValue, String routeField) { + if (recordValue == null) { + return null; + } + // TODO audit this to see if we need to avoid catching it. + Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); + return routeValue == null ? null : routeValue.toString(); + } + } + + private class StaticRecordRouter extends RecordRouter { + WriterForTable writerForTable; + String routeField; + StaticRecordRouter(WriterForTable writerForTable, String routeField) { + this.writerForTable = writerForTable; + } + @Override + public void write(SinkRecord record) { + if (routeField == null) { + // route to all tables + config + .tables() + .forEach( + tableName -> { + writerForTable.write(tableName, record, false); + }); + + } else { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + config + .tables() + .forEach( + tableName -> + config + .tableConfig(tableName) + .routeRegex() + .ifPresent( + regex -> { + if (regex.matcher(routeValue).matches()) { + writerForTable.write(tableName, record, false); + } + })); + } + } + } + } + + private class DynamicRecordRouter extends RecordRouter { + + WriterForTable writerForTable; + String routeField; + + DynamicRecordRouter(WriterForTable writerForTable, String routeField) { + this.writerForTable = writerForTable; + this.routeField = routeField; + } + @Override + public void write(SinkRecord record) { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + String tableName = routeValue.toLowerCase(); + writerForTable.write(tableName, record, true); + } + } + } public Worker( IcebergSinkConfig config, @@ -72,10 +189,24 @@ public Worker( clientFactory); this.config = config; - this.writerFactory = writerFactory; this.context = context; this.controlGroupId = config.controlGroupId(); this.writers = Maps.newHashMap(); + + WriterForTable writerForTable; + if (config.deadLetterTableEnabled()) { + writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers); + } else { + writerForTable = new BaseWriterForTable(writerFactory, this.writers); + } + + if (config.dynamicTablesEnabled()) { + Preconditions.checkNotNull(config.tablesRouteField(), "Route field cannot be null with dynamic routing"); + recordRouter = new DynamicRecordRouter(writerForTable, config.tablesRouteField()); + } else { + recordRouter = new StaticRecordRouter(writerForTable, config.tablesRouteField()); + } + this.sourceOffsets = Maps.newHashMap(); } @@ -178,67 +309,7 @@ private void save(SinkRecord record) { new TopicPartition(record.topic(), record.kafkaPartition()), new Offset(record.kafkaOffset() + 1, record.timestamp())); - if (config.dynamicTablesEnabled()) { - routeRecordDynamically(record); - } else { - routeRecordStatically(record); - } + recordRouter.write(record); } - private void routeRecordStatically(SinkRecord record) { - String routeField = config.tablesRouteField(); - - if (routeField == null) { - // route to all tables - config - .tables() - .forEach( - tableName -> { - writerForTable(tableName, record, false).write(record); - }); - - } else { - String routeValue = extractRouteValue(record.value(), routeField); - if (routeValue != null) { - config - .tables() - .forEach( - tableName -> - config - .tableConfig(tableName) - .routeRegex() - .ifPresent( - regex -> { - if (regex.matcher(routeValue).matches()) { - writerForTable(tableName, record, false).write(record); - } - })); - } - } - } - - private void routeRecordDynamically(SinkRecord record) { - String routeField = config.tablesRouteField(); - Preconditions.checkNotNull(routeField, "Route field cannot be null with dynamic routing"); - - String routeValue = extractRouteValue(record.value(), routeField); - if (routeValue != null) { - String tableName = routeValue.toLowerCase(); - writerForTable(tableName, record, true).write(record); - } - } - - private String extractRouteValue(Object recordValue, String routeField) { - if (recordValue == null) { - return null; - } - Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); - return routeValue == null ? null : routeValue.toString(); - } - - private RecordWriter writerForTable( - String tableName, SinkRecord sample, boolean ignoreMissingTable) { - return writers.computeIfAbsent( - tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); - } } From e87d8207d842296cba7b4e982a83bb216a824446 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 4 Apr 2024 11:30:00 -0600 Subject: [PATCH 02/32] next steps --- .../main/java/io/tabular/iceberg/connect/channel/Worker.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 056a0285..04bd7443 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -93,6 +93,9 @@ private static class DeadLetterWriterForTable implements WriterForTable { @Override public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { + // TODO + // dig out the correct part of the message here based on the dead letter table shape + RecordWriter writer = writers.computeIfAbsent( tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); try { From b062c51ad86f4859ab859e4826025931437951e8 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 5 Apr 2024 10:32:39 -0600 Subject: [PATCH 03/32] moved transform in --- gradle/libs.versions.toml | 1 + kafka-connect-transforms/build.gradle | 1 + .../connect/transforms/ErrorTransform.java | 462 ++++++++++++++++ .../transforms/ErrorTransformTest.java | 493 ++++++++++++++++++ .../transforms/TestStringTransform.java | 89 ++++ .../iceberg/connect/IcebergSinkConfig.java | 7 + .../iceberg/connect/channel/Worker.java | 202 +++++-- .../iceberg/connect/data/IcebergWriter.java | 16 +- .../connect/data/IcebergWriterFactory.java | 13 +- 9 files changed, 1230 insertions(+), 54 deletions(-) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java create mode 100644 kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java create mode 100644 kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/TestStringTransform.java diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2420248d..4f88011c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -48,6 +48,7 @@ http-client = { module = "org.apache.httpcomponents.client5:httpclient5", versi junit-api = { module = "org.junit.jupiter:junit-jupiter-api", version.ref = "junit-ver" } junit-engine = { module = "org.junit.jupiter:junit-jupiter-engine", version.ref = "junit-ver" } junit-params = { module = "org.junit.jupiter:junit-jupiter-params", version.ref = "junit-ver" } +kafka-connect-runtime = {module = "org.apache.kafka:connect-runtime", version.ref = "kafka-ver"} mockito = "org.mockito:mockito-core:4.8.1" testcontainers = { module = "org.testcontainers:testcontainers", version.ref = "testcontainers-ver" } testcontainers-kafka = { module = "org.testcontainers:kafka", version.ref = "testcontainers-ver" } diff --git a/kafka-connect-transforms/build.gradle b/kafka-connect-transforms/build.gradle index effec24e..b04aea68 100644 --- a/kafka-connect-transforms/build.gradle +++ b/kafka-connect-transforms/build.gradle @@ -8,6 +8,7 @@ dependencies { testImplementation libs.mockito testImplementation libs.assertj + testRuntimeOnly libs.kafka.connect.runtime } configurations { diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java new file mode 100644 index 00000000..567db38f --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -0,0 +1,462 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.transforms.Transformation; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +/** + * Wraps key, value, header converters and SMTs in order to catch exceptions. Failed records are + * converted into a standard struct and sent to the connector to be put into Iceberg + * + *

MUST ONLY BE USED with `value.converter`, `key.converter`, and `header.converter` set to + * "org.apache.kafka.connect.converters.ByteArrayConverter" which can not be validated from within + * this SMT + * + *

Actual value converter, key converter, and header converter are configured on the + * `transforms.xxx` key where xxx is the name of this transform. See example for how properties are + * passed and namespaced + * + *

"transforms": "tab", "transforms.tab.type": + * "io.tabular.iceberg.connect.transform.managed.ManagedTransform", + * "transforms.tab.value.converter": "org.apache.kafka.connect.storage.StringConverter", + * "transforms.tab.value.converter.some_property: "...", "transforms.tab.key.converter": + * "org.apache.kafka.connect.storage.StringConverter", "transforms.tab.key.converter.some_property": + * "...", "transforms.tab.smts" : "some.java.class,some.other.java.class", + * "transforms.tab.smts.prop1" : "some_property_for_the_smts" + * + *

This should not be used with any other SMT. All SMTs should be added to "transforms.tab.smts". + * + *

It returns a special Map of String -> Object "original" : Map of String -> Object containing + * the key,value, and header bytes of the original message "transformed" : [null, Struct, Map, etc.] + * of whatever the deserialized record is (after transformation if SMTs are configured) + * + *

The original payload can be used in the Iceberg Connector if the record cannot be transformed + * to an Iceberg record so that the original kafka message can be stored in Iceberg at that point. + * + *

If any of the key, value, header deserializers or SMTs throw an exception a failed record is + * constructed that contains kafka metadata, exception/location information, and the original + * key/value/header bytes. + */ +public class ErrorTransform implements Transformation { + + public static class TransformInitializationException extends RuntimeException { + TransformInitializationException(String errorMessage) { + super(errorMessage); + } + + TransformInitializationException(String errorMessage, Throwable err) { + super(errorMessage, err); + } + } + + public static class PropsParser { + static Map apply(Map props, String target) { + return props.entrySet().stream() + .filter( + entry -> + (!Objects.equals(entry.getKey(), target)) && (entry.getKey().startsWith(target))) + .collect( + Collectors.toMap( + entry -> entry.getKey().replaceFirst("^" + target + ".", ""), + Map.Entry::getValue)); + } + } + + private static class DeserializedRecord { + private final SinkRecord record; + private final boolean failed; + + DeserializedRecord(SinkRecord record, boolean failed) { + this.record = record; + this.failed = failed; + } + + public SinkRecord getRecord() { + return record; + } + + public boolean isFailed() { + return failed; + } + } + + private abstract static class ExceptionHandler { + SinkRecord handle(SinkRecord original, Throwable error, String location) { + throw new java.lang.IllegalStateException("handle not implemented"); + } + + protected final SinkRecord failedRecord(SinkRecord original, Throwable error, String location) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + error.printStackTrace(pw); + String stackTrace = sw.toString(); + Struct struct = new Struct(FAILED_SCHEMA); + struct.put("topic", original.topic()); + struct.put("partition", original.kafkaPartition()); + struct.put("offset", original.kafkaOffset()); + struct.put("timestamp", original.timestamp()); + struct.put("location", location); + struct.put("exception", error.toString()); + struct.put("stack_trace", stackTrace); + struct.put("key_bytes", original.key()); + struct.put("value_bytes", original.value()); + + if (!original.headers().isEmpty()) { + List headers = serializedHeaders(original); + struct.put(HEADERS, headers); + } + + return original.newRecord( + original.topic(), + original.kafkaPartition(), + null, + null, + FAILED_SCHEMA, + struct, + original.timestamp()); + } + } + + private static class AllExceptions extends ExceptionHandler { + @Override + SinkRecord handle(SinkRecord original, Throwable error, String location) { + return failedRecord(original, error, location); + } + } + + private static final String PAYLOAD_KEY = "transformed"; + private static final String ORIGINAL_BYTES_KEY = "original"; + private static final String KEY_BYTES = "key"; + private static final String VALUE_BYTES = "value"; + private static final String HEADERS = "headers"; + private static final String KEY_CONVERTER = "key.converter"; + private static final String VALUE_CONVERTER = "value.converter"; + private static final String HEADER_CONVERTER = "header.converter"; + private static final String TRANSFORMATIONS = "smts"; + private static final String KEY_FAILURE = "KEY_CONVERTER"; + private static final String VALUE_FAILURE = "VALUE_CONVERTER"; + private static final String HEADER_FAILURE = "HEADER_CONVERTER"; + private static final String SMT_FAILURE = "SMT_FAILURE"; + static final Schema HEADER_ELEMENT_SCHEMA = + SchemaBuilder.struct() + .field("key", Schema.STRING_SCHEMA) + .field("value", Schema.OPTIONAL_BYTES_SCHEMA) + .optional() + .build(); + static final Schema HEADER_SCHEMA = SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); + static final Schema FAILED_SCHEMA = + SchemaBuilder.struct() + .name("failed_message") + .parameter("isFailed", "true") + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("location", Schema.STRING_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) + .field("exception", Schema.OPTIONAL_STRING_SCHEMA) + .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) + .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field(HEADERS, HEADER_SCHEMA) + .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) + .schema(); + + private ExceptionHandler errorHandler; + private List> smts; + private Function keyConverter; + private Function valueConverter; + private Function headerConverterFn; + + public static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + KEY_CONVERTER, + ConfigDef.Type.STRING, + "org.apache.kafka.connect.converters.ByteArrayConverter", + ConfigDef.Importance.MEDIUM, + "key.converter") + .define( + VALUE_CONVERTER, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + "value.converter") + .define( + HEADER_CONVERTER, + ConfigDef.Type.STRING, + "org.apache.kafka.connect.converters.ByteArrayConverter", + ConfigDef.Importance.MEDIUM, + "header.converter") + .define( + TRANSFORMATIONS, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "smts"); + + @Override + public SinkRecord apply(SinkRecord record) { + // tombstones returned as-is + if (record == null || record.value() == null) { + return record; + } + + DeserializedRecord deserialized = deserialize(record); + if (deserialized.isFailed()) { + return deserialized.getRecord(); + } + + SinkRecord transformedRecord = deserialized.getRecord(); + + for (Transformation smt : smts) { + try { + transformedRecord = smt.apply(transformedRecord); + if (transformedRecord == null) { + break; + } + } catch (Exception e) { + return errorHandler.handle(record, e, SMT_FAILURE); + } + } + // SMT could filter out messages + if (transformedRecord == null) { + return null; + } + + return newRecord(record, transformedRecord); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() {} + + /* + Kafka Connect filters the properties it passes to the SMT to + only the keys under the `transform.xxx` name. + */ + @SuppressWarnings("unchecked") + @Override + public void configure(Map props) { + SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + ClassLoader loader = this.getClass().getClassLoader(); + + if (Objects.equals( + config.getString(KEY_CONVERTER), + "org.apache.kafka.connect.converters.ByteArrayConverter")) { + keyConverter = record -> new SchemaAndValue(record.keySchema(), record.value()); + } else { + Converter converter = (Converter) loadClass(config.getString(KEY_CONVERTER), loader); + converter.configure(PropsParser.apply(props, KEY_CONVERTER), true); + keyConverter = record -> converter.toConnectData(record.topic(), (byte[]) record.key()); + } + + if (config.getString(VALUE_CONVERTER) == null) { + throw new TransformInitializationException( + "ManagedTransformWrapper cannot be used without a defined value converter"); + } else { + Converter converter = (Converter) loadClass(config.getString(VALUE_CONVERTER), loader); + converter.configure(PropsParser.apply(props, VALUE_CONVERTER), false); + valueConverter = record -> converter.toConnectData(record.topic(), (byte[]) record.value()); + } + + HeaderConverter headerConverter; + + if (Objects.equals( + config.getString(HEADER_CONVERTER), + "org.apache.kafka.connect.converters.ByteArrayConverter")) { + try (HeaderConverter converter = + (HeaderConverter) + loadClass("org.apache.kafka.connect.converters.ByteArrayConverter", loader)) { + converter.configure(PropsParser.apply(props, HEADER_CONVERTER)); + headerConverter = converter; + } catch (Exception e) { + throw new TransformInitializationException( + String.format( + "Error loading header converter class %s", config.getString(HEADER_CONVERTER)), + e); + } + headerConverterFn = ConnectRecord::headers; + } else { + try (HeaderConverter converter = + (HeaderConverter) loadClass(config.getString(HEADER_CONVERTER), loader)) { + converter.configure(PropsParser.apply(props, HEADER_CONVERTER)); + headerConverter = converter; + } catch (Exception e) { + throw new TransformInitializationException( + String.format( + "Error loading header converter class %s", config.getString(HEADER_CONVERTER)), + e); + } + + headerConverterFn = + record -> { + Headers newHeaders = new ConnectHeaders(); + Headers recordHeaders = record.headers(); + if (recordHeaders != null) { + String topic = record.topic(); + for (Header recordHeader : recordHeaders) { + SchemaAndValue schemaAndValue = + headerConverter.toConnectHeader( + topic, recordHeader.key(), (byte[]) recordHeader.value()); + newHeaders.add(recordHeader.key(), schemaAndValue); + } + } + return newHeaders; + }; + } + + if (config.getString(TRANSFORMATIONS) == null) { + smts = Lists.newArrayList(); + } else { + + smts = + Arrays.stream(config.getString(TRANSFORMATIONS).split(",")) + .map(className -> loadClass(className, loader)) + .map(obj -> (Transformation) obj) + .peek(smt -> smt.configure(PropsParser.apply(props, TRANSFORMATIONS))) + .collect(Collectors.toList()); + } + + errorHandler = new AllExceptions(); + } + + private Object loadClass(String name, ClassLoader loader) { + if (name == null || name.isEmpty()) { + throw new TransformInitializationException("cannot initialize empty class"); + } + Object obj; + try { + Class clazz = Class.forName(name, true, loader); + obj = clazz.getDeclaredConstructor().newInstance(); + } catch (Exception e) { + throw new TransformInitializationException( + String.format("could not initialize class %s", name), e); + } + return obj; + } + + private DeserializedRecord deserialize(SinkRecord record) { + SchemaAndValue keyData; + SchemaAndValue valueData; + Headers newHeaders; + + try { + keyData = keyConverter.apply(record); + } catch (Exception e) { + return new DeserializedRecord(errorHandler.handle(record, e, KEY_FAILURE), true); + } + + try { + valueData = valueConverter.apply(record); + } catch (Exception e) { + return new DeserializedRecord(errorHandler.handle(record, e, VALUE_FAILURE), true); + } + try { + newHeaders = headerConverterFn.apply(record); + } catch (Exception e) { + return new DeserializedRecord(errorHandler.handle(record, e, HEADER_FAILURE), true); + } + + return new DeserializedRecord( + record.newRecord( + record.topic(), + record.kafkaPartition(), + keyData.schema(), + keyData.value(), + valueData.schema(), + valueData.value(), + record.timestamp(), + newHeaders), + false); + } + + private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { + Map bytes = Maps.newHashMap(); + + if (original.key() != null) { + bytes.put(KEY_BYTES, original.key()); + } + if (original.value() == null) { + throw new IllegalStateException("newRecord called with null value for record.value"); + } + + if (!original.headers().isEmpty()) { + bytes.put(HEADERS, serializedHeaders(original)); + } + + bytes.put(VALUE_BYTES, original.value()); + + Map result = Maps.newHashMap(); + result.put(PAYLOAD_KEY, transformed); + result.put(ORIGINAL_BYTES_KEY, bytes); + + return transformed.newRecord( + transformed.topic(), + transformed.kafkaPartition(), + null, + null, + null, + result, + transformed.timestamp(), + transformed.headers()); + } + + /** + * No way to get back the original Kafka header bytes. We instead have an array with elements of + * {"key": String, "value": bytes} for each header. This can be converted back into a Kafka + * Connect header by the user later, and further converted into Kafka RecordHeaders to be put back + * into a ProducerRecord to create the original headers on the Kafka record. + * + * @param original record where headers are still byte array values + * @return Struct for an Array that can be put into Iceberg + */ + private static List serializedHeaders(SinkRecord original) { + List headers = Lists.newArrayList(); + for (Header header : original.headers()) { + Struct headerStruct = new Struct(HEADER_ELEMENT_SCHEMA); + headerStruct.put("key", header.key()); + headerStruct.put("value", header.value()); + headers.add(headerStruct); + } + return headers; + } +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java new file mode 100644 index 00000000..a51d0480 --- /dev/null +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -0,0 +1,493 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +public class ErrorTransformTest { + + private static final String TOPIC = "some-topic"; + private static final int PARTITION = 3; + private static final long OFFSET = 100; + private static final long TIMESTAMP = 1000; + private static final String KEY_STRING = "key"; + private static final String VALUE_STRING = "value"; + private static final String KEY_JSON = "{\"key\": \"blah\"}"; + private static final String VALUE_JSON = "{\"a\": 1, \"b\": \"b\"}"; + private static final String BYTE_ARRAY_CONVERTER = + "org.apache.kafka.connect.converters.ByteArrayConverter"; + private static final String JSON_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; + private static final String STRING_CONVERTER = "org.apache.kafka.connect.storage.StringConverter"; + + private Headers stringAsByteHeaders() { + Headers headers = new ConnectHeaders(); + headers.add( + "h1", new SchemaAndValue(Schema.BYTES_SCHEMA, "h1".getBytes(StandardCharsets.UTF_8))); + return headers; + } + + private SinkRecord createRecord(String key, String value, Headers headers) { + byte[] valueBytes = (value == null) ? null : value.getBytes(StandardCharsets.UTF_8); + byte[] keyBytes = (key == null) ? null : key.getBytes(StandardCharsets.UTF_8); + + return new SinkRecord( + TOPIC, + PARTITION, + null, + keyBytes, + null, + valueBytes, + OFFSET, + TIMESTAMP, + TimestampType.CREATE_TIME, + headers); + } + + @Test + @DisplayName( + "It should deserialize using the supplied converters into the custom SinkRecord shape with original/transformed result") + public void deserialize() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + STRING_CONVERTER, + "header.converter.converter.type", + "header")); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders())); + + assertThat(result.keySchema()).isNull(); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + + // can't assert on map due to byte array equality + Map original = (Map) value.get("original"); + byte[] valueBytes = (byte[]) original.get("value"); + byte[] keyBytes = (byte[]) original.get("key"); + assertThat(valueBytes).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(keyBytes).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + + assertThat(original.get("headers")).isInstanceOf(ArrayList.class); + List resultHeaders = (List) (original.get("headers")); + assertThat(resultHeaders).isNotEmpty(); + + Struct headerElement = (Struct) resultHeaders.get(0); + assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat((byte[]) headerElement.get("value")) + .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + + assertThat(value.get("transformed")).isInstanceOf(SinkRecord.class); + SinkRecord transformed = (SinkRecord) value.get("transformed"); + assertThat(transformed.value()).isEqualTo(VALUE_STRING); + assertThat(transformed.valueSchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); + assertThat(transformed.key()).isEqualTo(KEY_STRING); + assertThat(transformed.keySchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); + assertThat(transformed.topic()).isEqualTo(TOPIC); + assertThat(transformed.kafkaPartition()).isEqualTo(PARTITION); + assertThat(transformed.kafkaOffset()).isEqualTo(OFFSET); + + ConnectHeaders expectedHeaders = new ConnectHeaders(); + expectedHeaders.add("h1", new SchemaAndValue(Schema.OPTIONAL_STRING_SCHEMA, "h1")); + assertThat(transformed.headers()).isEqualTo(expectedHeaders); + } + } + + @Test + @DisplayName("It should not have a key entry for original bytes if the key was null") + public void nullKey() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of("value.converter", STRING_CONVERTER, "key.converter", STRING_CONVERTER)); + SinkRecord result = smt.apply(createRecord(null, VALUE_STRING, null)); + + assertThat(result.keySchema()).isNull(); + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + Map original = (Map) value.get("original"); + + assertThat(original.containsKey("value")).isTrue(); + assertThat(original.containsKey("key")).isFalse(); + } + } + + @Test + @DisplayName("It should apply the configured nested SMT transforms") + public void smt() { + try (ErrorTransform smt = new ErrorTransform()) { + + String transformString = "_transformed"; + + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "smts", + "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", + "smts.transform_text", + transformString)); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); + + assertThat(result.value()).isInstanceOf(Map.class); + Map value = (Map) result.value(); + + assertThat(value.get("transformed")).isInstanceOf(SinkRecord.class); + SinkRecord transformed = (SinkRecord) value.get("transformed"); + // each transformer appends _transformed to the original value + // we are configured with two transform appenders + assertThat(transformed.value()).isEqualTo(VALUE_STRING + "_transformed_transformed"); + assertThat(transformed.key()).isEqualTo(KEY_STRING); + assertThat(transformed.keySchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); + assertThat(transformed.topic()).isEqualTo(TOPIC); + assertThat(transformed.kafkaPartition()).isEqualTo(PARTITION); + assertThat(transformed.kafkaOffset()).isEqualTo(OFFSET); + assertThat(transformed.headers()).isEqualTo(new ConnectHeaders()); + } + } + + @Test + @DisplayName("Tombstone records should be returned as-is") + public void tombstone() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of("value.converter", STRING_CONVERTER, "key.converter", STRING_CONVERTER)); + SinkRecord record = createRecord(null, null, null); + assertThat(smt.apply(record)).isSameAs(record); + } + } + + @Test + @DisplayName("Should return null if SMT filters out message") + public void nullFilteredBySMT() { + try (ErrorTransform smt = new ErrorTransform()) { + + String transformString = "_transformed"; + + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "smts", + "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", + "smts.transform_text", + transformString, + "smts.null", + "true")); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); + + assertThat(result).isNull(); + } + } + + @Test + @DisplayName("Should return failed SinkRecord if key deserializer fails") + public void keyFailed() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + JSON_CONVERTER, + "header.converter", + STRING_CONVERTER, + "header.converter.converter.type", + "header")); + + String malformedKey = "{\"malformed_json\"\"\"{}{}{}{}**"; + SinkRecord result = + smt.apply(createRecord(malformedKey, VALUE_STRING, stringAsByteHeaders())); + assertThat(result.keySchema()).isNull(); + assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("topic")).isEqualTo(TOPIC); + assertThat(value.get("partition")).isEqualTo(PARTITION); + assertThat(value.get("offset")).isEqualTo(OFFSET); + assertThat(value.get("location")).isEqualTo("KEY_CONVERTER"); + assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); + assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); + assertThat((byte[]) value.get("key_bytes")) + .isEqualTo(malformedKey.getBytes(StandardCharsets.UTF_8)); + assertThat((byte[]) value.get("value_bytes")) + .isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + + assertThat(value.get("headers")).isInstanceOf(List.class); + List resultHeaders = (List) (value.get("headers")); + assertThat(resultHeaders).isNotEmpty(); + Struct headerElement = (Struct) resultHeaders.get(0); + assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat((byte[]) headerElement.get("value")) + .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + } + } + + @Test + @DisplayName("Should return failed SinkRecord if value deserializer fails") + public void valueFailed() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of( + "value.converter", + JSON_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + STRING_CONVERTER, + "header.converter.converter.type", + "header")); + + String malformedValue = "{\"malformed_json\"\"\"{}{}{}{}**"; + SinkRecord result = + smt.apply(createRecord(KEY_STRING, malformedValue, stringAsByteHeaders())); + assertThat(result.keySchema()).isNull(); + assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("topic")).isEqualTo(TOPIC); + assertThat(value.get("partition")).isEqualTo(PARTITION); + assertThat(value.get("offset")).isEqualTo(OFFSET); + assertThat(value.get("location")).isEqualTo("VALUE_CONVERTER"); + assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); + assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); + assertThat((byte[]) value.get("key_bytes")) + .isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat((byte[]) value.get("value_bytes")) + .isEqualTo(malformedValue.getBytes(StandardCharsets.UTF_8)); + + assertThat(value.get("headers")).isInstanceOf(List.class); + List resultHeaders = (ArrayList) (value.get("headers")); + assertThat(resultHeaders).isNotEmpty(); + Struct headerElement = (Struct) resultHeaders.get(0); + assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat((byte[]) headerElement.get("value")) + .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + } + } + + @Test + @DisplayName("Should return failed SinkRecord if header deserializer fails") + public void headerFailed() { + String malformedValue = "{\"malformed_json\"\"\"{}{}{}{}**"; + Headers headers = new ConnectHeaders(); + headers.add( + "h1", + new SchemaAndValue(Schema.BYTES_SCHEMA, malformedValue.getBytes(StandardCharsets.UTF_8))); + + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + JSON_CONVERTER, + "header.converter.schemas.enable", + "false", + "header.converter.converter.type", + "header")); + + SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, headers); + SinkRecord result = smt.apply(record); + assertThat(result.keySchema()).isNull(); + assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("topic")).isEqualTo(TOPIC); + assertThat(value.get("partition")).isEqualTo(PARTITION); + assertThat(value.get("offset")).isEqualTo(OFFSET); + assertThat(value.get("location")).isEqualTo("HEADER_CONVERTER"); + assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); + assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); + assertThat((byte[]) value.get("key_bytes")) + .isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat((byte[]) value.get("value_bytes")) + .isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + + assertThat(value.get("headers")).isInstanceOf(List.class); + List resultHeaders = (ArrayList) (value.get("headers")); + assertThat(resultHeaders).isNotEmpty(); + Struct headerElement = (Struct) resultHeaders.get(0); + assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat((byte[]) headerElement.get("value")) + .isEqualTo(malformedValue.getBytes(StandardCharsets.UTF_8)); + } + } + + @Test + @DisplayName("Should return failed SinkRecord if SMT fails") + public void smtFailed() { + try (ErrorTransform smt = new ErrorTransform()) { + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + STRING_CONVERTER, + "header.converter.converter.type", + "header", + "smts", + "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", + "smts.throw", + "true")); + + SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders()); + SinkRecord result = smt.apply(record); + assertThat(result.keySchema()).isNull(); + assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct value = (Struct) result.value(); + assertThat(value.get("topic")).isEqualTo(TOPIC); + assertThat(value.get("partition")).isEqualTo(PARTITION); + assertThat(value.get("offset")).isEqualTo(OFFSET); + assertThat(value.get("location")).isEqualTo("SMT_FAILURE"); + assertThat(((String) value.get("stack_trace")).contains("smt failure")).isTrue(); + assertThat(((String) value.get("exception")).contains("smt failure")).isTrue(); + assertThat((byte[]) value.get("key_bytes")) + .isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat((byte[]) value.get("value_bytes")) + .isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + + assertThat(value.get("headers")).isInstanceOf(List.class); + List resultHeaders = (ArrayList) (value.get("headers")); + assertThat(resultHeaders).isNotEmpty(); + Struct headerElement = (Struct) resultHeaders.get(0); + assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat((byte[]) headerElement.get("value")) + .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + } + } + + @Test + @DisplayName("Should throw if runtime classes cannot be dynamically loaded or configured") + public void shouldThrowClassLoader() { + try (ErrorTransform smt = new ErrorTransform()) { + assertThrows( + ErrorTransform.TransformInitializationException.class, + () -> smt.configure(ImmutableMap.of("value.converter", ""))); + } + + try (ErrorTransform smt = new ErrorTransform()) { + assertThrows( + ErrorTransform.TransformInitializationException.class, + () -> smt.configure(ImmutableMap.of("value.converter", "some_bogus_class"))); + } + + try (ErrorTransform smt = new ErrorTransform()) { + assertThrows( + ErrorTransform.TransformInitializationException.class, + () -> + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + "some_bogus_class", + "header.converter.converter.type", + "header", + "smts", + "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", + "smts.throw", + "true"))); + } + + try (ErrorTransform smt = new ErrorTransform()) { + assertThrows( + ErrorTransform.TransformInitializationException.class, + () -> + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + STRING_CONVERTER, + "header.converter.converter.type", + "header", + "smts", + "some_bogus_smt", + "smts.throw", + "true"))); + } + try (ErrorTransform smt = new ErrorTransform()) { + // throws because the header converter fails when .configure is called + assertThrows( + ErrorTransform.TransformInitializationException.class, + () -> + smt.configure( + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "header.converter", + STRING_CONVERTER))); + } + } + + @Test + @DisplayName("PropsParser should return an empty map for keys that do not match target") + public void propsParserEmptyMap() { + Map input = ImmutableMap.of("some.key", "some.value"); + assertThat(ErrorTransform.PropsParser.apply(input, "missing")).isEmpty(); + } + + @Test + @DisplayName( + "PropsParser should return Map with keys matching target stripped of the target prefix") + public void keysMatching() { + Map input = + ImmutableMap.of("value.converter", "some.class.here", "value.converter.prop", "some.prop"); + assertThat(ErrorTransform.PropsParser.apply(input, "value.converter")) + .isEqualTo(ImmutableMap.of("prop", "some.prop")); + } +} diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/TestStringTransform.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/TestStringTransform.java new file mode 100644 index 00000000..6d48e2e0 --- /dev/null +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/TestStringTransform.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.transforms.Transformation; + +/* Appends values to record.values that are strings, useful for testing SMT control flow */ +public class TestStringTransform implements Transformation { + + private String text; + private boolean returnNull; + + private boolean shouldThrow; + + @Override + public synchronized SinkRecord apply(SinkRecord record) { + if (shouldThrow) { + throw new RuntimeException("smt failure"); + } + if (record.value() == null) { + return record; + } + if (returnNull) { + return null; + } + String newValue; + if (record.value() instanceof String) { + newValue = (record.value()) + text; + } else { + throw new IllegalArgumentException("record.value is not a string"); + } + return record.newRecord( + record.topic(), + record.kafkaPartition(), + record.keySchema(), + record.key(), + record.valueSchema(), + newValue, + record.timestamp()); + } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } + + @Override + public void close() {} + + @Override + public void configure(Map map) { + if (map.get("transform_text") != null) { + text = (String) map.get("transform_text"); + } else { + text = "default"; + } + + if (map.get("null") != null) { + returnNull = Boolean.parseBoolean((String) map.get("null")); + } else { + returnNull = false; + } + + if (map.get("throw") != null) { + shouldThrow = Boolean.parseBoolean((String) map.get("throw")); + } else { + shouldThrow = false; + } + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index c278a904..1af0923b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,6 +83,8 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; + private static final String DEAD_LETTER_TABLE_SUFFIX_PROP = "iceberg.tables.deadletter.suffix"; + private static final String DEAD_LETTER_TABLE_SUFFIX_DEFAULT = "__dlt"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -338,6 +340,11 @@ public boolean deadLetterTableEnabled() { return false; } + // TODO handle the default + public String deadLetterTableSuffix() { + return getString(DEAD_LETTER_TABLE_SUFFIX_PROP); + } + public String tablesRouteField() { return getString(TABLES_ROUTE_FIELD_PROP); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 04bd7443..7440e27c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -46,6 +46,9 @@ import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -53,7 +56,6 @@ public class Worker extends Channel { private final IcebergSinkConfig config; -// private final IcebergWriterFactory writerFactory; private final SinkTaskContext context; private final String controlGroupId; private final Map writers; @@ -76,40 +78,148 @@ private static class BaseWriterForTable implements WriterForTable { @Override public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { - writers.computeIfAbsent( - tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)).write(sample); - + writers + .computeIfAbsent( + tableName, + notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)) + .write(sample); } } private static class DeadLetterWriterForTable implements WriterForTable { + private static final String PAYLOAD_KEY = "transformed"; + private static final String ORIGINAL_BYTES_KEY = "original"; + private static final String KEY_BYTES = "key"; + private static final String VALUE_BYTES = "value"; + private static final String HEADERS = "headers"; private final IcebergWriterFactory writerFactory; private final Map writers; - - DeadLetterWriterForTable(IcebergWriterFactory writerFactory, Map writers) { + private final String deadLetterTableSuffix; + + static final Schema HEADER_ELEMENT_SCHEMA = + SchemaBuilder.struct() + .field("key", Schema.STRING_SCHEMA) + .field("value", Schema.OPTIONAL_BYTES_SCHEMA) + .optional() + .build(); + static final Schema HEADER_SCHEMA = + SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); + static final Schema FAILED_SCHEMA = + SchemaBuilder.struct() + .name("failed_message") + .parameter("isFailed", "true") + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("location", Schema.STRING_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) + .field("exception", Schema.OPTIONAL_STRING_SCHEMA) + .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) + .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field(HEADERS, HEADER_SCHEMA) + .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) + .schema(); + + DeadLetterWriterForTable( + IcebergWriterFactory writerFactory, + Map writers, + IcebergSinkConfig config) { this.writerFactory = writerFactory; this.writers = writers; + this.deadLetterTableSuffix = config.deadLetterTableSuffix(); } + @SuppressWarnings("unchecked") @Override public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { - // TODO - // dig out the correct part of the message here based on the dead letter table shape - - RecordWriter writer = writers.computeIfAbsent( - tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); - try { - writer.write(sample); - } catch (Exception e) { - // dig out the bytes - // generate the table name (topic + _dlt) ? - // writers.computeIfAbsent - // write the message. + if (sample.value() != null) { + if (sample.value() instanceof Map) { + RecordWriter writer; + Map payload = (Map) sample.value(); + SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); + if (isFailed(transformed)) { + String deadLetterTableName = deadLetterTableName(sample); + + // TODO here I know the tableName so need to dig out the struct and inject the + // target_table parameter + + writer = + writers.computeIfAbsent( + deadLetterTableName, + notUsed -> + writerFactory.createWriter( + deadLetterTableName, transformed, ignoreMissingTable)); + } else { + writer = + writers.computeIfAbsent( + tableName, + notUsed -> + writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); + } + try { + writer.write(sample); + } catch (Exception e) { + String deadLetterTableName = deadLetterTableName(sample); + SinkRecord newRecord = failedRecord(tableName, sample); + // TODO figure out if I can make this more granular + writers + .computeIfAbsent( + deadLetterTableName, + notUsed -> writerFactory.createWriter(tableName, newRecord, ignoreMissingTable)) + .write(newRecord); + } + } else { + throw new IllegalArgumentException( + "Record not in format expected for dead letter handling"); + } } } + + private boolean isFailed(SinkRecord record) { + return "true".equals(record.valueSchema().parameters().get("isFailed")); + } + + private String deadLetterTableName(SinkRecord record) { + // TODO maybe make this a config option? + return record.topic() + deadLetterTableSuffix; + } + + @SuppressWarnings("unchecked") + private SinkRecord failedRecord(String targetTable, SinkRecord record) { + Map payload = (Map) record.value(); + Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); + Struct struct = new Struct(FAILED_SCHEMA); + Object keyBytes = bytes.get(KEY_BYTES); + Object valueBytes = bytes.get(VALUE_BYTES); + Object headers = bytes.get(HEADERS); + if (bytes.get(KEY_BYTES) != null) { + struct.put("key_bytes", keyBytes); + } + if (valueBytes != null) { + struct.put("value_bytes", valueBytes); + } + if (headers != null) { + struct.put(HEADERS, headers); + } + struct.put("topic", record.topic()); + struct.put("partition", record.kafkaPartition()); + struct.put("offset", record.kafkaOffset()); + struct.put("record_timestamp", record.timestamp()); + struct.put("target_table", targetTable); + + return record.newRecord( + record.topic(), + record.kafkaPartition(), + null, + null, + FAILED_SCHEMA, + struct, + record.timestamp()); + } } - private abstract class RecordRouter { + private abstract static class RecordRouter { void write(SinkRecord record) {} protected String extractRouteValue(Object recordValue, String routeField) { @@ -123,52 +233,56 @@ protected String extractRouteValue(Object recordValue, String routeField) { } private class StaticRecordRouter extends RecordRouter { - WriterForTable writerForTable; - String routeField; + private final WriterForTable writerForTable; + private final String routeField; + StaticRecordRouter(WriterForTable writerForTable, String routeField) { this.writerForTable = writerForTable; + this.routeField = routeField; } + @Override public void write(SinkRecord record) { if (routeField == null) { // route to all tables config - .tables() - .forEach( - tableName -> { - writerForTable.write(tableName, record, false); - }); + .tables() + .forEach( + tableName -> { + writerForTable.write(tableName, record, false); + }); } else { String routeValue = extractRouteValue(record.value(), routeField); if (routeValue != null) { config - .tables() - .forEach( - tableName -> - config - .tableConfig(tableName) - .routeRegex() - .ifPresent( - regex -> { - if (regex.matcher(routeValue).matches()) { - writerForTable.write(tableName, record, false); - } - })); + .tables() + .forEach( + tableName -> + config + .tableConfig(tableName) + .routeRegex() + .ifPresent( + regex -> { + if (regex.matcher(routeValue).matches()) { + writerForTable.write(tableName, record, false); + } + })); } } } } - private class DynamicRecordRouter extends RecordRouter { + private static class DynamicRecordRouter extends RecordRouter { - WriterForTable writerForTable; - String routeField; + private final WriterForTable writerForTable; + private final String routeField; DynamicRecordRouter(WriterForTable writerForTable, String routeField) { this.writerForTable = writerForTable; this.routeField = routeField; } + @Override public void write(SinkRecord record) { String routeValue = extractRouteValue(record.value(), routeField); @@ -198,13 +312,14 @@ public Worker( WriterForTable writerForTable; if (config.deadLetterTableEnabled()) { - writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers); + writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers, config); } else { writerForTable = new BaseWriterForTable(writerFactory, this.writers); } if (config.dynamicTablesEnabled()) { - Preconditions.checkNotNull(config.tablesRouteField(), "Route field cannot be null with dynamic routing"); + Preconditions.checkNotNull( + config.tablesRouteField(), "Route field cannot be null with dynamic routing"); recordRouter = new DynamicRecordRouter(writerForTable, config.tablesRouteField()); } else { recordRouter = new StaticRecordRouter(writerForTable, config.tablesRouteField()); @@ -314,5 +429,4 @@ private void save(SinkRecord record) { recordRouter.write(record); } - } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java index e9edf0e1..9b81034c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java @@ -59,15 +59,13 @@ private void initNewWriter() { public void write(SinkRecord record) { try { // TODO: config to handle tombstones instead of always ignoring? - if (record.value() != null) { - Record row = convertToRow(record); - String cdcField = config.tablesCdcField(); - if (cdcField == null) { - writer.write(row); - } else { - Operation op = extractCdcOperation(record.value(), cdcField); - writer.write(new RecordWrapper(row, op)); - } + Record row = convertToRow(record); + String cdcField = config.tablesCdcField(); + if (cdcField == null) { + writer.write(row); + } else { + Operation op = extractCdcOperation(record.value(), cdcField); + writer.write(new RecordWrapper(row, op)); } } catch (Exception e) { throw new DataException( diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index a11d1cf1..f4f01c20 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -21,6 +21,7 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import java.util.List; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; @@ -41,9 +42,19 @@ public class IcebergWriterFactory { private final Catalog catalog; private final IcebergSinkConfig config; + private final Predicate shouldAutoCreate; + public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; this.config = config; + + if (config.autoCreateEnabled()) { + shouldAutoCreate = (unused) -> true; + } else if (config.dynamicTablesEnabled()) { + shouldAutoCreate = (tableName) -> tableName.endsWith(config.deadLetterTableSuffix()); + } else { + shouldAutoCreate = (unused) -> false; + } } public RecordWriter createWriter( @@ -53,7 +64,7 @@ public RecordWriter createWriter( try { table = catalog.loadTable(identifier); } catch (NoSuchTableException nst) { - if (config.autoCreateEnabled()) { + if (shouldAutoCreate.test(tableName)) { table = autoCreateTable(tableName, sample); } else if (ignoreMissingTable) { return new RecordWriter() {}; From 9b7f6b03875d34e4cdf5825824200c7367dfde46 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 7 Apr 2024 19:27:32 -0600 Subject: [PATCH 04/32] more refactoring --- kafka-connect-deadletter/build.gradle | 30 +++ .../connect/deadletter/DeadLetterUtils.java | 179 ++++++++++++ kafka-connect-transforms/build.gradle | 1 + .../connect/transforms/ErrorTransform.java | 96 +------ .../transforms/ErrorTransformTest.java | 9 +- kafka-connect/build.gradle | 1 + .../iceberg/connect/IcebergSinkConfig.java | 30 ++- .../iceberg/connect/channel/Worker.java | 254 +++++++++++------- .../iceberg/connect/channel/WorkerTest.java | 33 +++ settings.gradle | 3 + 10 files changed, 443 insertions(+), 193 deletions(-) create mode 100644 kafka-connect-deadletter/build.gradle create mode 100644 kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java diff --git a/kafka-connect-deadletter/build.gradle b/kafka-connect-deadletter/build.gradle new file mode 100644 index 00000000..edfe399a --- /dev/null +++ b/kafka-connect-deadletter/build.gradle @@ -0,0 +1,30 @@ +plugins { + id "java-test-fixtures" +} + +dependencies { + implementation libs.iceberg.core + implementation libs.iceberg.common + implementation libs.iceberg.guava + implementation libs.avro + compileOnly libs.bundles.kafka.connect + + + testImplementation libs.junit.api + testRuntimeOnly libs.junit.engine + + testImplementation libs.mockito + testImplementation libs.assertj + + testFixturesImplementation libs.iceberg.common + testFixturesImplementation libs.iceberg.core + testFixturesImplementation libs.avro +} + +publishing { + publications { + mavenJava(MavenPublication) { + from components.java + } + } +} diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java new file mode 100644 index 00000000..00e54913 --- /dev/null +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.deadletter; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.sink.SinkRecord; + +public class DeadLetterUtils { + + private DeadLetterUtils() { + throw new IllegalStateException("Should not be initialialized"); + } + + public static final String KEY_BYTES = "key"; + public static final String VALUE_BYTES = "value"; + public static final String PAYLOAD_KEY = "transformed"; + public static final String ORIGINAL_BYTES_KEY = "original"; + private static final String HEADERS = "headers"; + public static final Schema HEADER_ELEMENT_SCHEMA = + SchemaBuilder.struct() + .field("key", Schema.STRING_SCHEMA) + .field("value", Schema.OPTIONAL_BYTES_SCHEMA) + .optional() + .build(); + public static final Schema HEADER_SCHEMA = + SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); + public static final Schema FAILED_SCHEMA = + SchemaBuilder.struct() + .name("failed_message") + .parameter("isFailed", "true") + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("location", Schema.STRING_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) + .field("exception", Schema.OPTIONAL_STRING_SCHEMA) + .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) + .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field(HEADERS, HEADER_SCHEMA) + .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) + .schema(); + + public static String stackTrace(Throwable error) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + error.printStackTrace(pw); + return sw.toString(); + } + + public static class Values { + // expect byte[] + private final Object keyBytes; + // expect byte[] + private final Object valueBytes; + // expect List + private final Object headers; + + public Values(Object keyBytes, Object valueBytes, Object headers) { + this.keyBytes = keyBytes; + this.valueBytes = valueBytes; + this.headers = headers; + } + + public Object getKeyBytes() { + return keyBytes; + } + + public Object getValueBytes() { + return valueBytes; + } + + public Object getHeaders() { + return headers; + } + } + + public static SinkRecord failedRecord(SinkRecord original, Throwable error, String location) { + List headers = null; + if (!original.headers().isEmpty()) { + headers = DeadLetterUtils.serializedHeaders(original); + } + Values values = new Values(original.key(), original.value(), headers); + return failedRecord(original, values, error, location, null); + } + + private static SinkRecord failedRecord( + SinkRecord original, Values values, Throwable error, String location, String targetTable) { + + Struct struct = new Struct(FAILED_SCHEMA); + struct.put("topic", original.topic()); + struct.put("partition", original.kafkaPartition()); + struct.put("offset", original.kafkaOffset()); + struct.put("timestamp", original.timestamp()); + struct.put("location", location); + struct.put("exception", error.toString()); + String stack = stackTrace(error); + if (!stack.isEmpty()) { + struct.put("stack_trace", stackTrace(error)); + } + if (values.getKeyBytes() != null) { + struct.put("key_bytes", values.getKeyBytes()); + } + if (values.getValueBytes() != null) { + struct.put("value_bytes", values.getValueBytes()); + } + if (values.getHeaders() != null) { + struct.put(HEADERS, values.getHeaders()); + } + if (targetTable != null) { + struct.put("target_table", targetTable); + } + + return original.newRecord( + original.topic(), + original.kafkaPartition(), + null, + null, + FAILED_SCHEMA, + struct, + original.timestamp()); + } + + /** + * No way to get back the original Kafka header bytes. We instead have an array with elements of + * {"key": String, "value": bytes} for each header. This can be converted back into a Kafka + * Connect header by the user later, and further converted into Kafka RecordHeaders to be put back + * into a ProducerRecord to create the original headers on the Kafka record. + * + * @param original record where headers are still byte array values + * @return Struct for an Array that can be put into Iceberg + */ + public static List serializedHeaders(SinkRecord original) { + List headers = Lists.newArrayList(); + for (Header header : original.headers()) { + Struct headerStruct = new Struct(HEADER_ELEMENT_SCHEMA); + headerStruct.put("key", header.key()); + headerStruct.put("value", header.value()); + headers.add(headerStruct); + } + return headers; + } + + @SuppressWarnings("unchecked") + public static SinkRecord mapToFailedRecord( + String targetTable, SinkRecord record, String location, Throwable error) { + Map payload = (Map) record.value(); + Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); + Object keyBytes = bytes.get(KEY_BYTES); + Object valueBytes = bytes.get(VALUE_BYTES); + Object headers = bytes.get(HEADERS); + Values values = new Values(keyBytes, valueBytes, headers); + return failedRecord(record, values, error, location, targetTable); + } +} diff --git a/kafka-connect-transforms/build.gradle b/kafka-connect-transforms/build.gradle index b04aea68..7ea42f31 100644 --- a/kafka-connect-transforms/build.gradle +++ b/kafka-connect-transforms/build.gradle @@ -1,4 +1,5 @@ dependencies { + implementation project(":iceberg-kafka-connect-deadletter") implementation libs.iceberg.guava implementation libs.slf4j compileOnly libs.bundles.kafka.connect diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index 567db38f..b87b4478 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -18,8 +18,7 @@ */ package io.tabular.iceberg.connect.transforms; -import java.io.PrintWriter; -import java.io.StringWriter; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -30,10 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; -import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; @@ -123,50 +119,15 @@ private abstract static class ExceptionHandler { SinkRecord handle(SinkRecord original, Throwable error, String location) { throw new java.lang.IllegalStateException("handle not implemented"); } - - protected final SinkRecord failedRecord(SinkRecord original, Throwable error, String location) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - error.printStackTrace(pw); - String stackTrace = sw.toString(); - Struct struct = new Struct(FAILED_SCHEMA); - struct.put("topic", original.topic()); - struct.put("partition", original.kafkaPartition()); - struct.put("offset", original.kafkaOffset()); - struct.put("timestamp", original.timestamp()); - struct.put("location", location); - struct.put("exception", error.toString()); - struct.put("stack_trace", stackTrace); - struct.put("key_bytes", original.key()); - struct.put("value_bytes", original.value()); - - if (!original.headers().isEmpty()) { - List headers = serializedHeaders(original); - struct.put(HEADERS, headers); - } - - return original.newRecord( - original.topic(), - original.kafkaPartition(), - null, - null, - FAILED_SCHEMA, - struct, - original.timestamp()); - } } private static class AllExceptions extends ExceptionHandler { @Override SinkRecord handle(SinkRecord original, Throwable error, String location) { - return failedRecord(original, error, location); + return DeadLetterUtils.failedRecord(original, error, location); } } - private static final String PAYLOAD_KEY = "transformed"; - private static final String ORIGINAL_BYTES_KEY = "original"; - private static final String KEY_BYTES = "key"; - private static final String VALUE_BYTES = "value"; private static final String HEADERS = "headers"; private static final String KEY_CONVERTER = "key.converter"; private static final String VALUE_CONVERTER = "value.converter"; @@ -176,29 +137,6 @@ SinkRecord handle(SinkRecord original, Throwable error, String location) { private static final String VALUE_FAILURE = "VALUE_CONVERTER"; private static final String HEADER_FAILURE = "HEADER_CONVERTER"; private static final String SMT_FAILURE = "SMT_FAILURE"; - static final Schema HEADER_ELEMENT_SCHEMA = - SchemaBuilder.struct() - .field("key", Schema.STRING_SCHEMA) - .field("value", Schema.OPTIONAL_BYTES_SCHEMA) - .optional() - .build(); - static final Schema HEADER_SCHEMA = SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); - static final Schema FAILED_SCHEMA = - SchemaBuilder.struct() - .name("failed_message") - .parameter("isFailed", "true") - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("location", Schema.STRING_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) - .field("exception", Schema.OPTIONAL_STRING_SCHEMA) - .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) - .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field(HEADERS, HEADER_SCHEMA) - .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) - .schema(); private ExceptionHandler errorHandler; private List> smts; @@ -413,21 +351,21 @@ private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { Map bytes = Maps.newHashMap(); if (original.key() != null) { - bytes.put(KEY_BYTES, original.key()); + bytes.put(DeadLetterUtils.KEY_BYTES, original.key()); } if (original.value() == null) { throw new IllegalStateException("newRecord called with null value for record.value"); } if (!original.headers().isEmpty()) { - bytes.put(HEADERS, serializedHeaders(original)); + bytes.put(HEADERS, DeadLetterUtils.serializedHeaders(original)); } - bytes.put(VALUE_BYTES, original.value()); + bytes.put(DeadLetterUtils.VALUE_BYTES, original.value()); Map result = Maps.newHashMap(); - result.put(PAYLOAD_KEY, transformed); - result.put(ORIGINAL_BYTES_KEY, bytes); + result.put(DeadLetterUtils.PAYLOAD_KEY, transformed); + result.put(DeadLetterUtils.ORIGINAL_BYTES_KEY, bytes); return transformed.newRecord( transformed.topic(), @@ -439,24 +377,4 @@ private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { transformed.timestamp(), transformed.headers()); } - - /** - * No way to get back the original Kafka header bytes. We instead have an array with elements of - * {"key": String, "value": bytes} for each header. This can be converted back into a Kafka - * Connect header by the user later, and further converted into Kafka RecordHeaders to be put back - * into a ProducerRecord to create the original headers on the Kafka record. - * - * @param original record where headers are still byte array values - * @return Struct for an Array that can be put into Iceberg - */ - private static List serializedHeaders(SinkRecord original) { - List headers = Lists.newArrayList(); - for (Header header : original.headers()) { - Struct headerStruct = new Struct(HEADER_ELEMENT_SCHEMA); - headerStruct.put("key", header.key()); - headerStruct.put("value", header.value()); - headers.add(headerStruct); - } - return headers; - } } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java index a51d0480..0b81524e 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -237,7 +238,7 @@ public void keyFailed() { SinkRecord result = smt.apply(createRecord(malformedKey, VALUE_STRING, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); @@ -281,7 +282,7 @@ public void valueFailed() { SinkRecord result = smt.apply(createRecord(KEY_STRING, malformedValue, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); @@ -332,7 +333,7 @@ public void headerFailed() { SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, headers); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); @@ -379,7 +380,7 @@ public void smtFailed() { SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders()); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(ErrorTransform.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 4f6337c4..759e1eed 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -1,5 +1,6 @@ dependencies { implementation project(":iceberg-kafka-connect-events") + implementation project(":iceberg-kafka-connect-deadletter") implementation libs.bundles.iceberg implementation libs.bundles.jackson implementation libs.avro diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 1af0923b..11634252 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,8 +83,12 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; + private static final String USE_DEAD_LETTER_TABLE_PROP = "iceberg.tables.deadletter"; private static final String DEAD_LETTER_TABLE_SUFFIX_PROP = "iceberg.tables.deadletter.suffix"; private static final String DEAD_LETTER_TABLE_SUFFIX_DEFAULT = "__dlt"; + + private static final String DEAD_LETTER_TABLE_NAMESPACE = "iceberg.tables.deadletter.namespace"; + private static final String DEAD_LETTER_TABLE_NAMESPACE_DEFAULT = "dead_letter"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -239,6 +243,24 @@ private static ConfigDef newConfigDef() { null, Importance.MEDIUM, "Coordinator threads to use for table commits, default is (cores * 2)"); + configDef.define( + USE_DEAD_LETTER_TABLE_PROP, + Type.BOOLEAN, + false, + Importance.MEDIUM, + "Handle dead letter table payloads. Must use ErrorTransform SMT"); + configDef.define( + DEAD_LETTER_TABLE_SUFFIX_PROP, + Type.STRING, + DEAD_LETTER_TABLE_SUFFIX_DEFAULT, + Importance.MEDIUM, + "If using dead letter table, suffix to append to table to generate dead letter table name"); + configDef.define( + DEAD_LETTER_TABLE_NAMESPACE, + Type.STRING, + DEAD_LETTER_TABLE_NAMESPACE_DEFAULT, + Importance.MEDIUM, + "If using dead letter table, namespace to put the tables with undetermined destination"); return configDef; } @@ -335,16 +357,18 @@ public boolean dynamicTablesEnabled() { return getBoolean(TABLES_DYNAMIC_PROP); } - // TODO public boolean deadLetterTableEnabled() { - return false; + return getBoolean(USE_DEAD_LETTER_TABLE_PROP); } - // TODO handle the default public String deadLetterTableSuffix() { return getString(DEAD_LETTER_TABLE_SUFFIX_PROP); } + public String deadLetterTopicNamespace() { + return getString(DEAD_LETTER_TABLE_NAMESPACE); + } + public String tablesRouteField() { return getString(TABLES_ROUTE_FIELD_PROP); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 7440e27c..1aec6a11 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -27,6 +27,7 @@ import io.tabular.iceberg.connect.data.RecordWriter; import io.tabular.iceberg.connect.data.Utilities; import io.tabular.iceberg.connect.data.WriterResult; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; import io.tabular.iceberg.connect.events.CommitResponsePayload; @@ -46,8 +47,6 @@ import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; @@ -62,8 +61,10 @@ public class Worker extends Channel { private final Map sourceOffsets; private final RecordRouter recordRouter; - private interface WriterForTable { + public interface WriterForTable { void write(String tableName, SinkRecord sample, boolean ignoreMissingTable); + + void writeFailed(String namespace, SinkRecord sample, String location, Throwable error); } private static class BaseWriterForTable implements WriterForTable { @@ -84,42 +85,26 @@ public void write(String tableName, SinkRecord sample, boolean ignoreMissingTabl notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)) .write(sample); } + + @Override + public void writeFailed(String namespace, SinkRecord sample, String location, Throwable error) { + throw new IllegalArgumentException("BaseWriterForTable cannot write failed records", error); + } } - private static class DeadLetterWriterForTable implements WriterForTable { + public static class DeadLetterWriterForTable implements WriterForTable { private static final String PAYLOAD_KEY = "transformed"; private static final String ORIGINAL_BYTES_KEY = "original"; private static final String KEY_BYTES = "key"; private static final String VALUE_BYTES = "value"; private static final String HEADERS = "headers"; + + private static final String ICEBERG_TRANSFORMATION_LOCATION = "ICEBERG_TRANSFORM"; private final IcebergWriterFactory writerFactory; private final Map writers; private final String deadLetterTableSuffix; - static final Schema HEADER_ELEMENT_SCHEMA = - SchemaBuilder.struct() - .field("key", Schema.STRING_SCHEMA) - .field("value", Schema.OPTIONAL_BYTES_SCHEMA) - .optional() - .build(); - static final Schema HEADER_SCHEMA = - SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); - static final Schema FAILED_SCHEMA = - SchemaBuilder.struct() - .name("failed_message") - .parameter("isFailed", "true") - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("location", Schema.STRING_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) - .field("exception", Schema.OPTIONAL_STRING_SCHEMA) - .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) - .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field(HEADERS, HEADER_SCHEMA) - .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) - .schema(); + private final String deadLetterTopicNamespace; DeadLetterWriterForTable( IcebergWriterFactory writerFactory, @@ -128,6 +113,7 @@ private static class DeadLetterWriterForTable implements WriterForTable { this.writerFactory = writerFactory; this.writers = writers; this.deadLetterTableSuffix = config.deadLetterTableSuffix(); + this.deadLetterTopicNamespace = config.deadLetterTableSuffix(); } @SuppressWarnings("unchecked") @@ -139,10 +125,9 @@ public void write(String tableName, SinkRecord sample, boolean ignoreMissingTabl Map payload = (Map) sample.value(); SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); if (isFailed(transformed)) { - String deadLetterTableName = deadLetterTableName(sample); - - // TODO here I know the tableName so need to dig out the struct and inject the - // target_table parameter + String deadLetterTableName = deadLetterTableName(tableName); + Struct transformedStruct = (Struct) transformed.value(); + transformedStruct.put("target_table", tableName); writer = writers.computeIfAbsent( @@ -160,75 +145,113 @@ public void write(String tableName, SinkRecord sample, boolean ignoreMissingTabl try { writer.write(sample); } catch (Exception e) { - String deadLetterTableName = deadLetterTableName(sample); - SinkRecord newRecord = failedRecord(tableName, sample); - // TODO figure out if I can make this more granular + String deadLetterTableName = deadLetterTableName(tableName); + SinkRecord newRecord = + DeadLetterUtils.mapToFailedRecord( + tableName, sample, ICEBERG_TRANSFORMATION_LOCATION, e); writers .computeIfAbsent( deadLetterTableName, - notUsed -> writerFactory.createWriter(tableName, newRecord, ignoreMissingTable)) + notUsed -> + writerFactory.createWriter( + deadLetterTableName, newRecord, ignoreMissingTable)) .write(newRecord); } } else { - throw new IllegalArgumentException( - "Record not in format expected for dead letter handling"); + throw new IllegalArgumentException("Record not in format expected for dead letter table"); } } } + @Override + public void writeFailed(String namespace, SinkRecord sample, String location, Throwable error) { + String deadLetterTableName = deadLetterTableName(namespace, sample); + SinkRecord newRecord = DeadLetterUtils.mapToFailedRecord(null, sample, location, error); + writers + .computeIfAbsent( + deadLetterTableName, + notUsed -> writerFactory.createWriter(deadLetterTableName, newRecord, false)) + .write(newRecord); + } + private boolean isFailed(SinkRecord record) { return "true".equals(record.valueSchema().parameters().get("isFailed")); } - private String deadLetterTableName(SinkRecord record) { - // TODO maybe make this a config option? - return record.topic() + deadLetterTableSuffix; + private String deadLetterTableName(String namespace, SinkRecord record) { + return String.format("%s.%s_%s", namespace, record.topic(), deadLetterTableSuffix); } - @SuppressWarnings("unchecked") - private SinkRecord failedRecord(String targetTable, SinkRecord record) { - Map payload = (Map) record.value(); - Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); - Struct struct = new Struct(FAILED_SCHEMA); - Object keyBytes = bytes.get(KEY_BYTES); - Object valueBytes = bytes.get(VALUE_BYTES); - Object headers = bytes.get(HEADERS); - if (bytes.get(KEY_BYTES) != null) { - struct.put("key_bytes", keyBytes); + private String deadLetterTableName(String originalTableName) { + return originalTableName + deadLetterTableSuffix; + } + } + + private static class RouteExtractor { + public static class RouteValueResult { + private String value = null; + private Throwable error = null; + + RouteValueResult(String value) { + this.value = value; } - if (valueBytes != null) { - struct.put("value_bytes", valueBytes); + + RouteValueResult(Throwable error) { + this.error = error; } - if (headers != null) { - struct.put(HEADERS, headers); + + String getValue() { + return this.value; } - struct.put("topic", record.topic()); - struct.put("partition", record.kafkaPartition()); - struct.put("offset", record.kafkaOffset()); - struct.put("record_timestamp", record.timestamp()); - struct.put("target_table", targetTable); - - return record.newRecord( - record.topic(), - record.kafkaPartition(), - null, - null, - FAILED_SCHEMA, - struct, - record.timestamp()); - } - } - private abstract static class RecordRouter { - void write(SinkRecord record) {} + Throwable getError() { + return error; + } + } - protected String extractRouteValue(Object recordValue, String routeField) { + RouteValueResult extract(Object recordValue, String routeField) { if (recordValue == null) { return null; } - // TODO audit this to see if we need to avoid catching it. Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); - return routeValue == null ? null : routeValue.toString(); + return routeValue == null ? null : new RouteValueResult(routeValue.toString()); + } + } + + private static class ErrorHandlingRouteExtractor extends RouteExtractor { + @Override + RouteValueResult extract(Object recordValue, String routeField) { + RouteValueResult result; + try { + result = super.extract(recordValue, routeField); + } catch (Exception error) { + result = new RouteValueResult(error); + } + return result; + } + } + + private abstract static class RecordRouter { + + void write(SinkRecord record) {} + } + + private class ConfigRecordRouter extends RecordRouter { + private final WriterForTable writerForTable; + + ConfigRecordRouter(WriterForTable writerForTable) { + this.writerForTable = writerForTable; + } + + @Override + public void write(SinkRecord record) { + // route to all tables + config + .tables() + .forEach( + tableName -> { + writerForTable.write(tableName, record, false); + }); } } @@ -236,25 +259,27 @@ private class StaticRecordRouter extends RecordRouter { private final WriterForTable writerForTable; private final String routeField; - StaticRecordRouter(WriterForTable writerForTable, String routeField) { + private final RouteExtractor extractor; + + private final String deadLetterNamespace; + + StaticRecordRouter( + WriterForTable writerForTable, + String routeField, + RouteExtractor extractor, + String deadLetterNamespace) { this.writerForTable = writerForTable; this.routeField = routeField; + this.extractor = extractor; + this.deadLetterNamespace = deadLetterNamespace; } @Override public void write(SinkRecord record) { - if (routeField == null) { - // route to all tables - config - .tables() - .forEach( - tableName -> { - writerForTable.write(tableName, record, false); - }); - - } else { - String routeValue = extractRouteValue(record.value(), routeField); - if (routeValue != null) { + RouteExtractor.RouteValueResult routeValue = extractor.extract(record.value(), routeField); + if (routeValue != null) { + String route = routeValue.getValue(); + if (route != null) { config .tables() .forEach( @@ -264,31 +289,48 @@ public void write(SinkRecord record) { .routeRegex() .ifPresent( regex -> { - if (regex.matcher(routeValue).matches()) { + if (regex.matcher(route).matches()) { writerForTable.write(tableName, record, false); } })); + } else if (routeValue.getError() != null) { + writerForTable.writeFailed( + deadLetterNamespace, record, "ROUTE_EXTRACT", routeValue.getError()); } } } } private static class DynamicRecordRouter extends RecordRouter { - private final WriterForTable writerForTable; private final String routeField; + private final RouteExtractor extractor; - DynamicRecordRouter(WriterForTable writerForTable, String routeField) { + private final String deadLetterNamespace; + + DynamicRecordRouter( + WriterForTable writerForTable, + String routeField, + RouteExtractor extractor, + String deadLetterNamespace) { this.writerForTable = writerForTable; this.routeField = routeField; + this.extractor = extractor; + this.deadLetterNamespace = deadLetterNamespace; } @Override public void write(SinkRecord record) { - String routeValue = extractRouteValue(record.value(), routeField); + RouteExtractor.RouteValueResult routeValue = extractor.extract(record.value(), routeField); if (routeValue != null) { - String tableName = routeValue.toLowerCase(); - writerForTable.write(tableName, record, true); + String route = routeValue.getValue(); + if (route != null) { + String tableName = route.toLowerCase(); + writerForTable.write(tableName, record, true); + } else if (routeValue.getError() != null) { + writerForTable.writeFailed( + deadLetterNamespace, record, "ROUTE_EXTRACT", routeValue.getError()); + } } } } @@ -310,19 +352,37 @@ public Worker( this.controlGroupId = config.controlGroupId(); this.writers = Maps.newHashMap(); + RouteExtractor routeExtractor; + WriterForTable writerForTable; if (config.deadLetterTableEnabled()) { writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers, config); + routeExtractor = new ErrorHandlingRouteExtractor(); } else { writerForTable = new BaseWriterForTable(writerFactory, this.writers); + routeExtractor = new RouteExtractor(); } if (config.dynamicTablesEnabled()) { Preconditions.checkNotNull( config.tablesRouteField(), "Route field cannot be null with dynamic routing"); - recordRouter = new DynamicRecordRouter(writerForTable, config.tablesRouteField()); + recordRouter = + new DynamicRecordRouter( + writerForTable, + config.tablesRouteField(), + routeExtractor, + config.deadLetterTopicNamespace()); } else { - recordRouter = new StaticRecordRouter(writerForTable, config.tablesRouteField()); + if (config.tablesRouteField() == null) { + recordRouter = new ConfigRecordRouter(writerForTable); + } else { + recordRouter = + new StaticRecordRouter( + writerForTable, + config.tablesRouteField(), + routeExtractor, + config.deadLetterTopicNamespace()); + } } this.sourceOffsets = Maps.newHashMap(); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index c7954900..e90e2e3a 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -26,6 +26,7 @@ import io.tabular.iceberg.connect.data.IcebergWriter; import io.tabular.iceberg.connect.data.IcebergWriterFactory; +import io.tabular.iceberg.connect.data.RecordWriter; import io.tabular.iceberg.connect.data.WriterResult; import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; @@ -33,17 +34,20 @@ import io.tabular.iceberg.connect.events.Event; import io.tabular.iceberg.connect.events.EventTestUtil; import io.tabular.iceberg.connect.events.EventType; +import java.util.List; import java.util.Map; import java.util.UUID; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; public class WorkerTest extends ChannelTestBase { @@ -116,4 +120,33 @@ private void workerTest(Map value) { // offset should be one more than the record offset assertThat(readyPayload.assignments().get(0).offset()).isEqualTo(1L); } + + private static class RecordingRecordWriter implements RecordWriter { + List seen = Lists.newArrayList(); + + public void write(SinkRecord record) { + seen.add(record); + } + } + + @Test + @DisplayName("DeadLetterWriterForTable should handle custom ErrorTransform payloads") + public void deadLetterWriterForTable() { + // RecordWriter writer = new RecordingRecordWriter(); + // + // IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + // when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + // + // Map writerMap = Maps.newHashMap(); + // + // Worker.DeadLetterWriterForTable writerForTable = new Worker.DeadLetterWriterForTable( + // writerFactory, + // writerMap, + // config + // ); + // + // SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, null, 0); + // writerForTable.write(, ); + + } } diff --git a/settings.gradle b/settings.gradle index 0418d2cf..9820da4e 100644 --- a/settings.gradle +++ b/settings.gradle @@ -9,3 +9,6 @@ project(":iceberg-kafka-connect-transforms").projectDir = file("kafka-connect-tr include "iceberg-kafka-connect-runtime" project(":iceberg-kafka-connect-runtime").projectDir = file("kafka-connect-runtime") + +include "iceberg-kafka-connect-deadletter" +project(":iceberg-kafka-connect-deadletter").projectDir = file("kafka-connect-deadletter") From c1b4de24f19a7995277f863a7fcaca16549471e7 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 7 Apr 2024 19:58:36 -0600 Subject: [PATCH 05/32] even more refactoring --- .../connect/deadletter/DeadLetterUtils.java | 19 +++ .../iceberg/connect/channel/Worker.java | 161 +++++++++--------- 2 files changed, 102 insertions(+), 78 deletions(-) diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index 00e54913..49fa9e5b 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -31,6 +31,25 @@ public class DeadLetterUtils { + public static class DeadLetterException extends RuntimeException { + private final String location; + private final Throwable error; + + public DeadLetterException(String location, Throwable error) { + super(error); + this.location = location; + this.error = error; + } + + public String getLocation() { + return location; + } + + public Throwable getError() { + return error; + } + } + private DeadLetterUtils() { throw new IllegalStateException("Should not be initialialized"); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 1aec6a11..3a3f981b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -187,50 +187,6 @@ private String deadLetterTableName(String originalTableName) { } } - private static class RouteExtractor { - public static class RouteValueResult { - private String value = null; - private Throwable error = null; - - RouteValueResult(String value) { - this.value = value; - } - - RouteValueResult(Throwable error) { - this.error = error; - } - - String getValue() { - return this.value; - } - - Throwable getError() { - return error; - } - } - - RouteValueResult extract(Object recordValue, String routeField) { - if (recordValue == null) { - return null; - } - Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); - return routeValue == null ? null : new RouteValueResult(routeValue.toString()); - } - } - - private static class ErrorHandlingRouteExtractor extends RouteExtractor { - @Override - RouteValueResult extract(Object recordValue, String routeField) { - RouteValueResult result; - try { - result = super.extract(recordValue, routeField); - } catch (Exception error) { - result = new RouteValueResult(error); - } - return result; - } - } - private abstract static class RecordRouter { void write(SinkRecord record) {} @@ -255,6 +211,28 @@ public void write(SinkRecord record) { } } + private static class ErrorHandlingRecordRouter extends RecordRouter { + private final RecordRouter underlying; + private final WriterForTable writerForTable; + private final String namespace; + + ErrorHandlingRecordRouter( + RecordRouter underlying, WriterForTable writerForTable, String namespace) { + this.underlying = underlying; + this.writerForTable = writerForTable; + this.namespace = namespace; + } + + @Override + public void write(SinkRecord record) { + try { + underlying.write(record); + } catch (DeadLetterUtils.DeadLetterException e) { + writerForTable.writeFailed(namespace, record, e.getLocation(), e.getError()); + } + } + } + private class StaticRecordRouter extends RecordRouter { private final WriterForTable writerForTable; private final String routeField; @@ -276,27 +254,21 @@ private class StaticRecordRouter extends RecordRouter { @Override public void write(SinkRecord record) { - RouteExtractor.RouteValueResult routeValue = extractor.extract(record.value(), routeField); + String routeValue = extractor.extract(record.value(), routeField); if (routeValue != null) { - String route = routeValue.getValue(); - if (route != null) { - config - .tables() - .forEach( - tableName -> - config - .tableConfig(tableName) - .routeRegex() - .ifPresent( - regex -> { - if (regex.matcher(route).matches()) { - writerForTable.write(tableName, record, false); - } - })); - } else if (routeValue.getError() != null) { - writerForTable.writeFailed( - deadLetterNamespace, record, "ROUTE_EXTRACT", routeValue.getError()); - } + config + .tables() + .forEach( + tableName -> + config + .tableConfig(tableName) + .routeRegex() + .ifPresent( + regex -> { + if (regex.matcher(routeValue).matches()) { + writerForTable.write(tableName, record, false); + } + })); } } } @@ -321,16 +293,10 @@ private static class DynamicRecordRouter extends RecordRouter { @Override public void write(SinkRecord record) { - RouteExtractor.RouteValueResult routeValue = extractor.extract(record.value(), routeField); + String routeValue = extractor.extract(record.value(), routeField); if (routeValue != null) { - String route = routeValue.getValue(); - if (route != null) { - String tableName = route.toLowerCase(); - writerForTable.write(tableName, record, true); - } else if (routeValue.getError() != null) { - writerForTable.writeFailed( - deadLetterNamespace, record, "ROUTE_EXTRACT", routeValue.getError()); - } + String tableName = routeValue.toLowerCase(); + writerForTable.write(tableName, record, true); } } } @@ -353,20 +319,21 @@ public Worker( this.writers = Maps.newHashMap(); RouteExtractor routeExtractor; + RecordRouter baseRecordRouter; WriterForTable writerForTable; if (config.deadLetterTableEnabled()) { writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers, config); - routeExtractor = new ErrorHandlingRouteExtractor(); + routeExtractor = new ErrorHandlingRouteExtractor(new DefaultRouteExtractor()); } else { writerForTable = new BaseWriterForTable(writerFactory, this.writers); - routeExtractor = new RouteExtractor(); + routeExtractor = new DefaultRouteExtractor(); } if (config.dynamicTablesEnabled()) { Preconditions.checkNotNull( config.tablesRouteField(), "Route field cannot be null with dynamic routing"); - recordRouter = + baseRecordRouter = new DynamicRecordRouter( writerForTable, config.tablesRouteField(), @@ -374,9 +341,9 @@ public Worker( config.deadLetterTopicNamespace()); } else { if (config.tablesRouteField() == null) { - recordRouter = new ConfigRecordRouter(writerForTable); + baseRecordRouter = new ConfigRecordRouter(writerForTable); } else { - recordRouter = + baseRecordRouter = new StaticRecordRouter( writerForTable, config.tablesRouteField(), @@ -384,10 +351,48 @@ public Worker( config.deadLetterTopicNamespace()); } } + if (config.deadLetterTableEnabled()) { + recordRouter = + new ErrorHandlingRecordRouter( + baseRecordRouter, writerForTable, config.deadLetterTopicNamespace()); + } else { + recordRouter = baseRecordRouter; + } this.sourceOffsets = Maps.newHashMap(); } + private interface RouteExtractor { + String extract(Object recordValue, String fieldName); + } + + private static class DefaultRouteExtractor implements RouteExtractor { + + public String extract(Object recordValue, String routeField) { + if (recordValue == null) { + return null; + } + Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); + return routeValue == null ? null : routeValue.toString(); + } + } + + private static class ErrorHandlingRouteExtractor implements RouteExtractor { + private final RouteExtractor underlying; + + ErrorHandlingRouteExtractor(RouteExtractor underlying) { + this.underlying = underlying; + } + + public String extract(Object recordValue, String routeField) { + try { + return underlying.extract(recordValue, routeField); + } catch (Exception error) { + throw new DeadLetterUtils.DeadLetterException("ROUTE_FIELD", error); + } + } + } + public void syncCommitOffsets() { Map offsets = commitOffsets().entrySet().stream() From 4e706e1968664a110f9eb29f63f0db8131287949 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 7 Apr 2024 20:00:25 -0600 Subject: [PATCH 06/32] cruft --- .../iceberg/connect/channel/Worker.java | 29 +++---------------- 1 file changed, 4 insertions(+), 25 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 3a3f981b..63265963 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -239,17 +239,10 @@ private class StaticRecordRouter extends RecordRouter { private final RouteExtractor extractor; - private final String deadLetterNamespace; - - StaticRecordRouter( - WriterForTable writerForTable, - String routeField, - RouteExtractor extractor, - String deadLetterNamespace) { + StaticRecordRouter(WriterForTable writerForTable, String routeField, RouteExtractor extractor) { this.writerForTable = writerForTable; this.routeField = routeField; this.extractor = extractor; - this.deadLetterNamespace = deadLetterNamespace; } @Override @@ -278,17 +271,11 @@ private static class DynamicRecordRouter extends RecordRouter { private final String routeField; private final RouteExtractor extractor; - private final String deadLetterNamespace; - DynamicRecordRouter( - WriterForTable writerForTable, - String routeField, - RouteExtractor extractor, - String deadLetterNamespace) { + WriterForTable writerForTable, String routeField, RouteExtractor extractor) { this.writerForTable = writerForTable; this.routeField = routeField; this.extractor = extractor; - this.deadLetterNamespace = deadLetterNamespace; } @Override @@ -334,21 +321,13 @@ public Worker( Preconditions.checkNotNull( config.tablesRouteField(), "Route field cannot be null with dynamic routing"); baseRecordRouter = - new DynamicRecordRouter( - writerForTable, - config.tablesRouteField(), - routeExtractor, - config.deadLetterTopicNamespace()); + new DynamicRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); } else { if (config.tablesRouteField() == null) { baseRecordRouter = new ConfigRecordRouter(writerForTable); } else { baseRecordRouter = - new StaticRecordRouter( - writerForTable, - config.tablesRouteField(), - routeExtractor, - config.deadLetterTopicNamespace()); + new StaticRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); } } if (config.deadLetterTableEnabled()) { From f9136d8a4e704b9361ad754b87032887c6f978a2 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 7 Apr 2024 20:02:24 -0600 Subject: [PATCH 07/32] even more cruft --- .../iceberg/connect/channel/Worker.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 63265963..1a84ad03 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -62,7 +62,7 @@ public class Worker extends Channel { private final RecordRouter recordRouter; public interface WriterForTable { - void write(String tableName, SinkRecord sample, boolean ignoreMissingTable); + void write(String tableName, SinkRecord record, boolean ignoreMissingTable); void writeFailed(String namespace, SinkRecord sample, String location, Throwable error); } @@ -78,12 +78,12 @@ private static class BaseWriterForTable implements WriterForTable { } @Override - public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { + public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { writers .computeIfAbsent( tableName, - notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)) - .write(sample); + notUsed -> writerFactory.createWriter(tableName, record, ignoreMissingTable)) + .write(record); } @Override @@ -104,8 +104,6 @@ public static class DeadLetterWriterForTable implements WriterForTable { private final Map writers; private final String deadLetterTableSuffix; - private final String deadLetterTopicNamespace; - DeadLetterWriterForTable( IcebergWriterFactory writerFactory, Map writers, @@ -113,16 +111,15 @@ public static class DeadLetterWriterForTable implements WriterForTable { this.writerFactory = writerFactory; this.writers = writers; this.deadLetterTableSuffix = config.deadLetterTableSuffix(); - this.deadLetterTopicNamespace = config.deadLetterTableSuffix(); } @SuppressWarnings("unchecked") @Override - public void write(String tableName, SinkRecord sample, boolean ignoreMissingTable) { - if (sample.value() != null) { - if (sample.value() instanceof Map) { + public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { + if (record.value() != null) { + if (record.value() instanceof Map) { RecordWriter writer; - Map payload = (Map) sample.value(); + Map payload = (Map) record.value(); SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); if (isFailed(transformed)) { String deadLetterTableName = deadLetterTableName(tableName); @@ -143,12 +140,12 @@ public void write(String tableName, SinkRecord sample, boolean ignoreMissingTabl writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); } try { - writer.write(sample); + writer.write(record); } catch (Exception e) { String deadLetterTableName = deadLetterTableName(tableName); SinkRecord newRecord = DeadLetterUtils.mapToFailedRecord( - tableName, sample, ICEBERG_TRANSFORMATION_LOCATION, e); + tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); writers .computeIfAbsent( deadLetterTableName, From d3905a59d6b794887e4798482b7e94a0917dd1ce Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 8 Apr 2024 08:18:10 -0600 Subject: [PATCH 08/32] tests --- .../iceberg/connect/channel/Worker.java | 70 ++++---- .../iceberg/connect/channel/WorkerTest.java | 169 +++++++++++++++--- 2 files changed, 186 insertions(+), 53 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 1a84ad03..9c1938b1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -94,11 +94,6 @@ public void writeFailed(String namespace, SinkRecord sample, String location, Th public static class DeadLetterWriterForTable implements WriterForTable { private static final String PAYLOAD_KEY = "transformed"; - private static final String ORIGINAL_BYTES_KEY = "original"; - private static final String KEY_BYTES = "key"; - private static final String VALUE_BYTES = "value"; - private static final String HEADERS = "headers"; - private static final String ICEBERG_TRANSFORMATION_LOCATION = "ICEBERG_TRANSFORM"; private final IcebergWriterFactory writerFactory; private final Map writers; @@ -117,13 +112,23 @@ public static class DeadLetterWriterForTable implements WriterForTable { @Override public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { if (record.value() != null) { + + RecordWriter writer = null; + SinkRecord recordToWrite = null; + if (record.value() instanceof Map) { - RecordWriter writer; Map payload = (Map) record.value(); SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); - if (isFailed(transformed)) { + writer = + writers.computeIfAbsent( + tableName, + notUsed -> + writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); + recordToWrite = transformed; + } else if (record.value() instanceof Struct) { + if (isFailed(record)) { String deadLetterTableName = deadLetterTableName(tableName); - Struct transformedStruct = (Struct) transformed.value(); + Struct transformedStruct = (Struct) record.value(); transformedStruct.put("target_table", tableName); writer = @@ -131,32 +136,28 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl deadLetterTableName, notUsed -> writerFactory.createWriter( - deadLetterTableName, transformed, ignoreMissingTable)); - } else { - writer = - writers.computeIfAbsent( - tableName, - notUsed -> - writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); - } - try { - writer.write(record); - } catch (Exception e) { - String deadLetterTableName = deadLetterTableName(tableName); - SinkRecord newRecord = - DeadLetterUtils.mapToFailedRecord( - tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); - writers - .computeIfAbsent( - deadLetterTableName, - notUsed -> - writerFactory.createWriter( - deadLetterTableName, newRecord, ignoreMissingTable)) - .write(newRecord); + deadLetterTableName, record, ignoreMissingTable)); + recordToWrite = record; } } else { throw new IllegalArgumentException("Record not in format expected for dead letter table"); } + + try { + writer.write(recordToWrite); + } catch (Exception e) { + String deadLetterTableName = deadLetterTableName(tableName); + SinkRecord newRecord = + DeadLetterUtils.mapToFailedRecord( + tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); + writers + .computeIfAbsent( + deadLetterTableName, + notUsed -> + writerFactory.createWriter( + deadLetterTableName, newRecord, ignoreMissingTable)) + .write(newRecord); + } } } @@ -172,7 +173,14 @@ public void writeFailed(String namespace, SinkRecord sample, String location, Th } private boolean isFailed(SinkRecord record) { - return "true".equals(record.valueSchema().parameters().get("isFailed")); + Map parameters = record.valueSchema().parameters(); + if (parameters != null) { + String isFailed = parameters.get("isFailed"); + if (isFailed != null) { + return isFailed.equals("true"); + } + } + return false; } private String deadLetterTableName(String namespace, SinkRecord record) { diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index e90e2e3a..cbb1441b 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -28,12 +28,14 @@ import io.tabular.iceberg.connect.data.IcebergWriterFactory; import io.tabular.iceberg.connect.data.RecordWriter; import io.tabular.iceberg.connect.data.WriterResult; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; import io.tabular.iceberg.connect.events.CommitResponsePayload; import io.tabular.iceberg.connect.events.Event; import io.tabular.iceberg.connect.events.EventTestUtil; import io.tabular.iceberg.connect.events.EventType; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.UUID; @@ -42,9 +44,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; import org.junit.jupiter.api.DisplayName; @@ -55,6 +62,57 @@ public class WorkerTest extends ChannelTestBase { private static final String TABLE_NAME = "db.tbl"; private static final String FIELD_NAME = "fld"; + private static class RecordingRecordWriter implements RecordWriter { + + private final boolean shouldThrowOnFirstRecord; + private int count = 0; + + RecordingRecordWriter(boolean shouldThrowOnFirstRecord) { + this.shouldThrowOnFirstRecord = shouldThrowOnFirstRecord; + } + + List written = Lists.newArrayList(); + + public void write(SinkRecord record) { + if (shouldThrowOnFirstRecord && count == 0) { + count += 1; + throw new IllegalArgumentException("test throw"); + } + written.add(record); + count += 1; + } + } + + private static final byte[] ORIGINAL_BYTES = + "{\"field\":\"success\"}".getBytes(StandardCharsets.UTF_8); + + private SinkRecord errorTransformSuccessRecord() { + Schema schema = + SchemaBuilder.struct() + .field("field", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .build(); + Struct struct = new Struct(schema); + struct.put("field", "success"); + Map map = Maps.newHashMap(); + SinkRecord success = + new SinkRecord( + "topic", 0, null, null, schema, struct, 100L, 1000L, TimestampType.CREATE_TIME); + map.put(DeadLetterUtils.PAYLOAD_KEY, success); + Map failed = Maps.newHashMap(); + failed.put(DeadLetterUtils.VALUE_BYTES, ORIGINAL_BYTES); + map.put(DeadLetterUtils.ORIGINAL_BYTES_KEY, failed); + return new SinkRecord( + "topic", 0, null, null, null, map, 100L, 1000L, TimestampType.CREATE_TIME); + } + + private SinkRecord errorTransformFailedRecord() { + SinkRecord originalRecord = + new SinkRecord( + "topic", 0, null, null, null, ORIGINAL_BYTES, 100L, 1000L, TimestampType.CREATE_TIME); + return DeadLetterUtils.failedRecord( + originalRecord, new IllegalArgumentException("test"), "location"); + } + @Test public void testStaticRoute() { when(config.tables()).thenReturn(ImmutableList.of(TABLE_NAME)); @@ -121,32 +179,99 @@ private void workerTest(Map value) { assertThat(readyPayload.assignments().get(0).offset()).isEqualTo(1L); } - private static class RecordingRecordWriter implements RecordWriter { - List seen = Lists.newArrayList(); + @Test + @DisplayName("DeadLetterWriterForTable should ignore null records") + public void deadLetterWriterForTableNullRecords() { + RecordingRecordWriter writer = new RecordingRecordWriter(false); - public void write(SinkRecord record) { - seen.add(record); - } + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, null, 0); + writerForTable.write(TABLE_NAME, nullRecord, true); + + assertThat(writer.written).isEmpty(); + } + + @Test + @DisplayName("DeadLetterWriterForTable should write successful ErrorTransformed records") + public void deadLetterWriterForTableSuccessErrorTransform() { + RecordingRecordWriter writer = new RecordingRecordWriter(false); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); + Schema expectedSchema = + SchemaBuilder.struct() + .field("field", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .build(); + assertThat(writer.written.size()).isEqualTo(1); + SinkRecord result = writer.written.get(0); + assertThat(result.valueSchema()).isEqualTo(expectedSchema); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct resultStruct = (Struct) result.value(); + assertThat(resultStruct.get("field")).isEqualTo("success"); + } + + @Test + @DisplayName("DeadLetterWriterForTable should write failed ErrorTransformed records") + public void deadLetterWriterForTableFailureErrorTransform() { + RecordingRecordWriter writer = new RecordingRecordWriter(false); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + writerForTable.write(TABLE_NAME, errorTransformFailedRecord(), true); + assertThat(writer.written.size()).isEqualTo(1); + SinkRecord result = writer.written.get(0); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct resultValue = (Struct) result.value(); + assertThat(resultValue.get("exception").toString().contains("test")).isTrue(); + String resultBytesAsString = + new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); + assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); } @Test - @DisplayName("DeadLetterWriterForTable should handle custom ErrorTransform payloads") - public void deadLetterWriterForTable() { - // RecordWriter writer = new RecordingRecordWriter(); - // - // IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - // when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); - // - // Map writerMap = Maps.newHashMap(); - // - // Worker.DeadLetterWriterForTable writerForTable = new Worker.DeadLetterWriterForTable( - // writerFactory, - // writerMap, - // config - // ); - // - // SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, null, 0); - // writerForTable.write(, ); + @DisplayName( + "DeadLetterWriterForTable should convert Iceberg Writer failures to failed SinkRecord") + public void deadLetterWriterForTableConvertWriterFailures() { + RecordingRecordWriter writer = new RecordingRecordWriter(true); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); + assertThat(writer.written.size()).isEqualTo(1); + SinkRecord result = writer.written.get(0); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct resultValue = (Struct) result.value(); + assertThat(resultValue.get("exception").toString().contains("test throw")).isTrue(); + String resultBytesAsString = + new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); + assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); } } From 18c79ba45d17f179437616732cb6de29c75e7ecb Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 8 Apr 2024 19:42:31 -0600 Subject: [PATCH 09/32] table create exceptions --- .../iceberg/connect/IcebergSinkConfig.java | 36 ++---- .../iceberg/connect/channel/Worker.java | 108 +++++++++--------- .../connect/data/IcebergWriterFactory.java | 92 ++++++++++++++- .../iceberg/connect/channel/WorkerTest.java | 102 +++++++++++++++-- 4 files changed, 245 insertions(+), 93 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 11634252..454641b1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,12 +83,7 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; - private static final String USE_DEAD_LETTER_TABLE_PROP = "iceberg.tables.deadletter"; - private static final String DEAD_LETTER_TABLE_SUFFIX_PROP = "iceberg.tables.deadletter.suffix"; - private static final String DEAD_LETTER_TABLE_SUFFIX_DEFAULT = "__dlt"; - - private static final String DEAD_LETTER_TABLE_NAMESPACE = "iceberg.tables.deadletter.namespace"; - private static final String DEAD_LETTER_TABLE_NAMESPACE_DEFAULT = "dead_letter"; + private static final String DEAD_LETTER_TABLE_PROP = "iceberg.tables.deadletter"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -244,23 +239,11 @@ private static ConfigDef newConfigDef() { Importance.MEDIUM, "Coordinator threads to use for table commits, default is (cores * 2)"); configDef.define( - USE_DEAD_LETTER_TABLE_PROP, - Type.BOOLEAN, - false, - Importance.MEDIUM, - "Handle dead letter table payloads. Must use ErrorTransform SMT"); - configDef.define( - DEAD_LETTER_TABLE_SUFFIX_PROP, + DEAD_LETTER_TABLE_PROP, Type.STRING, - DEAD_LETTER_TABLE_SUFFIX_DEFAULT, - Importance.MEDIUM, - "If using dead letter table, suffix to append to table to generate dead letter table name"); - configDef.define( - DEAD_LETTER_TABLE_NAMESPACE, - Type.STRING, - DEAD_LETTER_TABLE_NAMESPACE_DEFAULT, + null, Importance.MEDIUM, - "If using dead letter table, namespace to put the tables with undetermined destination"); + "If using ErrorTransform for Dead Letter Table, the db.name to write"); return configDef; } @@ -358,15 +341,12 @@ public boolean dynamicTablesEnabled() { } public boolean deadLetterTableEnabled() { - return getBoolean(USE_DEAD_LETTER_TABLE_PROP); - } - - public String deadLetterTableSuffix() { - return getString(DEAD_LETTER_TABLE_SUFFIX_PROP); + String table = getString(DEAD_LETTER_TABLE_PROP); + return table != null; } - public String deadLetterTopicNamespace() { - return getString(DEAD_LETTER_TABLE_NAMESPACE); + public String deadLetterTableName() { + return getString(DEAD_LETTER_TABLE_PROP); } public String tablesRouteField() { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 9c1938b1..ec3fd0a5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -42,6 +42,7 @@ import java.util.Map.Entry; import java.util.UUID; import java.util.concurrent.ExecutionException; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; @@ -64,7 +65,7 @@ public class Worker extends Channel { public interface WriterForTable { void write(String tableName, SinkRecord record, boolean ignoreMissingTable); - void writeFailed(String namespace, SinkRecord sample, String location, Throwable error); + void writeFailed(SinkRecord sample, String location, Throwable error, String targetTableName); } private static class BaseWriterForTable implements WriterForTable { @@ -87,7 +88,8 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl } @Override - public void writeFailed(String namespace, SinkRecord sample, String location, Throwable error) { + public void writeFailed( + SinkRecord sample, String location, Throwable error, String targetTableName) { throw new IllegalArgumentException("BaseWriterForTable cannot write failed records", error); } } @@ -97,7 +99,7 @@ public static class DeadLetterWriterForTable implements WriterForTable { private static final String ICEBERG_TRANSFORMATION_LOCATION = "ICEBERG_TRANSFORM"; private final IcebergWriterFactory writerFactory; private final Map writers; - private final String deadLetterTableSuffix; + private final String deadLetterTableName; DeadLetterWriterForTable( IcebergWriterFactory writerFactory, @@ -105,7 +107,9 @@ public static class DeadLetterWriterForTable implements WriterForTable { IcebergSinkConfig config) { this.writerFactory = writerFactory; this.writers = writers; - this.deadLetterTableSuffix = config.deadLetterTableSuffix(); + Preconditions.checkNotNull( + config.deadLetterTableName(), "Dead letter table name cannot be null"); + this.deadLetterTableName = config.deadLetterTableName().toLowerCase(); } @SuppressWarnings("unchecked") @@ -119,18 +123,24 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl if (record.value() instanceof Map) { Map payload = (Map) record.value(); SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); - writer = - writers.computeIfAbsent( - tableName, - notUsed -> - writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); - recordToWrite = transformed; + try { + writer = + writers.computeIfAbsent( + tableName, + notUsed -> + writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); + recordToWrite = transformed; + } catch (DeadLetterUtils.DeadLetterException error) { + writeFailed(record, error.getLocation(), error.getError(), tableName); + } } else if (record.value() instanceof Struct) { if (isFailed(record)) { - String deadLetterTableName = deadLetterTableName(tableName); Struct transformedStruct = (Struct) record.value(); transformedStruct.put("target_table", tableName); + // not sure I should wrap this? + // anything thrown here is a bug on our part, no? Someone has messed w/ the table? + // everything here should be valid at this point writer = writers.computeIfAbsent( deadLetterTableName, @@ -143,28 +153,30 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl throw new IllegalArgumentException("Record not in format expected for dead letter table"); } - try { - writer.write(recordToWrite); - } catch (Exception e) { - String deadLetterTableName = deadLetterTableName(tableName); - SinkRecord newRecord = - DeadLetterUtils.mapToFailedRecord( - tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); - writers - .computeIfAbsent( - deadLetterTableName, - notUsed -> - writerFactory.createWriter( - deadLetterTableName, newRecord, ignoreMissingTable)) - .write(newRecord); + if (recordToWrite != null) { + try { + writer.write(recordToWrite); + } catch (Exception e) { + SinkRecord newRecord = + DeadLetterUtils.mapToFailedRecord( + tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); + writers + .computeIfAbsent( + deadLetterTableName, + notUsed -> + writerFactory.createWriter( + deadLetterTableName, newRecord, ignoreMissingTable)) + .write(newRecord); + } } } } @Override - public void writeFailed(String namespace, SinkRecord sample, String location, Throwable error) { - String deadLetterTableName = deadLetterTableName(namespace, sample); - SinkRecord newRecord = DeadLetterUtils.mapToFailedRecord(null, sample, location, error); + public void writeFailed( + SinkRecord sample, String location, Throwable error, String targetTableName) { + SinkRecord newRecord = + DeadLetterUtils.mapToFailedRecord(targetTableName, sample, location, error); writers .computeIfAbsent( deadLetterTableName, @@ -182,14 +194,6 @@ private boolean isFailed(SinkRecord record) { } return false; } - - private String deadLetterTableName(String namespace, SinkRecord record) { - return String.format("%s.%s_%s", namespace, record.topic(), deadLetterTableSuffix); - } - - private String deadLetterTableName(String originalTableName) { - return originalTableName + deadLetterTableSuffix; - } } private abstract static class RecordRouter { @@ -197,35 +201,32 @@ private abstract static class RecordRouter { void write(SinkRecord record) {} } - private class ConfigRecordRouter extends RecordRouter { + private static class ConfigRecordRouter extends RecordRouter { private final WriterForTable writerForTable; + private final List tables; - ConfigRecordRouter(WriterForTable writerForTable) { + ConfigRecordRouter(WriterForTable writerForTable, List tables) { this.writerForTable = writerForTable; + this.tables = tables; } @Override public void write(SinkRecord record) { // route to all tables - config - .tables() - .forEach( - tableName -> { - writerForTable.write(tableName, record, false); - }); + tables.forEach( + tableName -> { + writerForTable.write(tableName, record, false); + }); } } private static class ErrorHandlingRecordRouter extends RecordRouter { private final RecordRouter underlying; private final WriterForTable writerForTable; - private final String namespace; - ErrorHandlingRecordRouter( - RecordRouter underlying, WriterForTable writerForTable, String namespace) { + ErrorHandlingRecordRouter(RecordRouter underlying, WriterForTable writerForTable) { this.underlying = underlying; this.writerForTable = writerForTable; - this.namespace = namespace; } @Override @@ -233,7 +234,7 @@ public void write(SinkRecord record) { try { underlying.write(record); } catch (DeadLetterUtils.DeadLetterException e) { - writerForTable.writeFailed(namespace, record, e.getLocation(), e.getError()); + writerForTable.writeFailed(record, e.getLocation(), e.getError(), null); } } } @@ -329,16 +330,17 @@ public Worker( new DynamicRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); } else { if (config.tablesRouteField() == null) { - baseRecordRouter = new ConfigRecordRouter(writerForTable); + // validate all table identifiers are valid, otherwise exception is thrown + // as this is an invalid config setting, not an error during processing + config.tables().forEach(TableIdentifier::of); + baseRecordRouter = new ConfigRecordRouter(writerForTable, config.tables()); } else { baseRecordRouter = new StaticRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); } } if (config.deadLetterTableEnabled()) { - recordRouter = - new ErrorHandlingRecordRouter( - baseRecordRouter, writerForTable, config.deadLetterTopicNamespace()); + recordRouter = new ErrorHandlingRecordRouter(baseRecordRouter, writerForTable); } else { recordRouter = baseRecordRouter; } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index f4f01c20..747afa63 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -19,14 +19,18 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.Tasks; @@ -35,6 +39,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +// TODO get the DLT options for partitioning from the config and look into how that works + public class IcebergWriterFactory { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); @@ -44,14 +50,81 @@ public class IcebergWriterFactory { private final Predicate shouldAutoCreate; + private TableIdentifierSupplier tableIdSupplier; + + private TableCreator tableCreator; + + private static class TableIdentifierSupplier { + TableIdentifier id(String name) { + return TableIdentifier.parse(name); + } + } + + private static class ErrorHandlingTableIdentifierSupplier extends TableIdentifierSupplier { + @Override + TableIdentifier id(String name) { + TableIdentifier tableId; + try { + tableId = super.id(name); + } catch (Exception error) { + throw new DeadLetterUtils.DeadLetterException("TABLE_IDENTIFIER", error); + } + return tableId; + } + } + + private static class TableCreator { + + private final Catalog catalog; + + TableCreator(Catalog catalog) { + this.catalog = catalog; + } + + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + Map properties) { + return catalog.createTable(identifier, schema, spec, properties); + } + } + + private class ErrorHandlingTableCreator extends TableCreator { + + ErrorHandlingTableCreator(Catalog catalog) { + super(catalog); + } + + @Override + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + Map properties) { + Table table; + try { + table = catalog.createTable(identifier, schema, spec, properties); + } catch (IllegalArgumentException | ValidationException error) { + throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); + } + return table; + } + } + public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; this.config = config; + this.tableCreator = new TableCreator(catalog); + this.tableIdSupplier = new TableIdentifierSupplier(); if (config.autoCreateEnabled()) { shouldAutoCreate = (unused) -> true; - } else if (config.dynamicTablesEnabled()) { - shouldAutoCreate = (tableName) -> tableName.endsWith(config.deadLetterTableSuffix()); + } else if (config.deadLetterTableEnabled()) { + String deadLetterTableName = config.deadLetterTableName().toLowerCase(); + shouldAutoCreate = (tableName) -> tableName.equals(deadLetterTableName); + this.tableCreator = new ErrorHandlingTableCreator(catalog); + this.tableIdSupplier = new ErrorHandlingTableIdentifierSupplier(); } else { shouldAutoCreate = (unused) -> false; } @@ -59,7 +132,9 @@ public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { public RecordWriter createWriter( String tableName, SinkRecord sample, boolean ignoreMissingTable) { - TableIdentifier identifier = TableIdentifier.parse(tableName); + + // this can fail. + TableIdentifier identifier = tableIdSupplier.id(tableName); Table table; try { table = catalog.loadTable(identifier); @@ -80,6 +155,10 @@ public RecordWriter createWriter( Table autoCreateTable(String tableName, SinkRecord sample) { StructType structType; if (sample.valueSchema() == null) { + + // TODO thnk this one through and step through it + // NOT sure if we need to tackle this one. + structType = SchemaUtils.inferIcebergType(sample.value(), config) .orElseThrow(() -> new DataException("Unable to create table from empty object")) @@ -89,9 +168,11 @@ Table autoCreateTable(String tableName, SinkRecord sample) { } org.apache.iceberg.Schema schema = new org.apache.iceberg.Schema(structType.fields()); - TableIdentifier identifier = TableIdentifier.parse(tableName); + + TableIdentifier identifier = tableIdSupplier.id(tableName); List partitionBy = config.tableConfig(tableName).partitionBy(); + PartitionSpec spec; try { spec = SchemaUtils.createPartitionSpec(schema, partitionBy); @@ -111,10 +192,11 @@ Table autoCreateTable(String tableName, SinkRecord sample) { .run( notUsed -> { try { + // provided the identifier is valid, we can ignore this one. result.set(catalog.loadTable(identifier)); } catch (NoSuchTableException e) { result.set( - catalog.createTable( + tableCreator.createTable( identifier, schema, partitionSpec, config.autoCreateProps())); } }); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index cbb1441b..79a80bb2 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -19,8 +19,7 @@ package io.tabular.iceberg.connect.channel; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -45,6 +44,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; @@ -56,11 +56,13 @@ import org.apache.kafka.connect.sink.SinkTaskContext; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentMatchers; public class WorkerTest extends ChannelTestBase { private static final String TABLE_NAME = "db.tbl"; private static final String FIELD_NAME = "fld"; + private static final String DEAD_LETTER_TABLE_NAME = "dead.tbl"; private static class RecordingRecordWriter implements RecordWriter { @@ -142,7 +144,9 @@ private void workerTest(Map value) { when(writer.complete()).thenReturn(ImmutableList.of(writeResult)); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); Worker worker = new Worker(config, clientFactory, writerFactory, context); worker.start(); @@ -182,10 +186,15 @@ private void workerTest(Map value) { @Test @DisplayName("DeadLetterWriterForTable should ignore null records") public void deadLetterWriterForTableNullRecords() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + RecordingRecordWriter writer = new RecordingRecordWriter(false); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); Map writerMap = Maps.newHashMap(); @@ -201,10 +210,15 @@ public void deadLetterWriterForTableNullRecords() { @Test @DisplayName("DeadLetterWriterForTable should write successful ErrorTransformed records") public void deadLetterWriterForTableSuccessErrorTransform() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + RecordingRecordWriter writer = new RecordingRecordWriter(false); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); Map writerMap = Maps.newHashMap(); @@ -227,10 +241,15 @@ public void deadLetterWriterForTableSuccessErrorTransform() { @Test @DisplayName("DeadLetterWriterForTable should write failed ErrorTransformed records") public void deadLetterWriterForTableFailureErrorTransform() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + RecordingRecordWriter writer = new RecordingRecordWriter(false); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); Map writerMap = Maps.newHashMap(); @@ -253,10 +272,14 @@ public void deadLetterWriterForTableFailureErrorTransform() { @DisplayName( "DeadLetterWriterForTable should convert Iceberg Writer failures to failed SinkRecord") public void deadLetterWriterForTableConvertWriterFailures() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); RecordingRecordWriter writer = new RecordingRecordWriter(true); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); Map writerMap = Maps.newHashMap(); @@ -273,5 +296,70 @@ public void deadLetterWriterForTableConvertWriterFailures() { String resultBytesAsString = new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); + // we successfully create the regular writer, but fail to write it, seeing both writers in the + // Map + assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(TABLE_NAME, DEAD_LETTER_TABLE_NAME)); + } + + @Test + @DisplayName( + "DeadLetterWriterForTable should fail when non-DeadLetterExceptions are thrown during writer creation") + public void deadLetterWriterThrowWriterCreation() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter( + TABLE_NAME, ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenThrow(new IllegalArgumentException("writer factory throws")); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + assertThrows( + IllegalArgumentException.class, + () -> writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true)); + } + + @Test + @DisplayName( + "DeadLetterWriterForTable should write failed SinkRecords when DeadLetterWriter exceptions are thrown during writer creation") + public void deadLetterWriterDeadLetterExceptionsAtCreation() { + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + + RecordingRecordWriter writer = new RecordingRecordWriter(false); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter( + ArgumentMatchers.eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenThrow( + new DeadLetterUtils.DeadLetterException( + "test", new IllegalArgumentException("creation throw"))); + when(writerFactory.createWriter( + ArgumentMatchers.eq(DEAD_LETTER_TABLE_NAME), + ArgumentMatchers.any(), + ArgumentMatchers.anyBoolean())) + .thenReturn(writer); + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); + + assertThat(writer.written.size()).isEqualTo(1); + SinkRecord result = writer.written.get(0); + assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.value()).isInstanceOf(Struct.class); + Struct resultValue = (Struct) result.value(); + assertThat(resultValue.get("stack_trace").toString().contains("creation throw")).isTrue(); + String resultBytesAsString = + new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); + assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); + // fail to create the original message, but do successfully create the dead letter table + assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(DEAD_LETTER_TABLE_NAME)); } } From 77aad4bc1f0af06c78674b80c6728478b3932f21 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 9 Apr 2024 11:47:11 -0600 Subject: [PATCH 10/32] introduces identifier column, more error handle wrapping --- .../connect/deadletter/DeadLetterUtils.java | 23 ++++-- .../connect/transforms/ErrorTransform.java | 2 +- .../iceberg/connect/channel/Worker.java | 9 ++- .../connect/data/IcebergWriterFactory.java | 80 +++++++++++++------ .../iceberg/connect/channel/WorkerTest.java | 2 +- 5 files changed, 81 insertions(+), 35 deletions(-) diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index 49fa9e5b..ac24f92b 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -54,6 +54,7 @@ private DeadLetterUtils() { throw new IllegalStateException("Should not be initialialized"); } + public static final String TYPE = "dlt"; public static final String KEY_BYTES = "key"; public static final String VALUE_BYTES = "value"; public static final String PAYLOAD_KEY = "transformed"; @@ -71,6 +72,7 @@ private DeadLetterUtils() { SchemaBuilder.struct() .name("failed_message") .parameter("isFailed", "true") + .field("identifier", Schema.OPTIONAL_STRING_SCHEMA) .field("topic", Schema.STRING_SCHEMA) .field("partition", Schema.INT32_SCHEMA) .field("offset", Schema.INT64_SCHEMA) @@ -82,6 +84,7 @@ private DeadLetterUtils() { .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) .field(HEADERS, HEADER_SCHEMA) .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) + .field("type", Schema.STRING_SCHEMA) .schema(); public static String stackTrace(Throwable error) { @@ -118,19 +121,28 @@ public Object getHeaders() { } } - public static SinkRecord failedRecord(SinkRecord original, Throwable error, String location) { + public static SinkRecord failedRecord( + SinkRecord original, Throwable error, String location, String identifier) { List headers = null; if (!original.headers().isEmpty()) { headers = DeadLetterUtils.serializedHeaders(original); } Values values = new Values(original.key(), original.value(), headers); - return failedRecord(original, values, error, location, null); + return failedRecord(original, values, error, location, null, identifier); } private static SinkRecord failedRecord( - SinkRecord original, Values values, Throwable error, String location, String targetTable) { + SinkRecord original, + Values values, + Throwable error, + String location, + String targetTable, + String identifier) { Struct struct = new Struct(FAILED_SCHEMA); + if (identifier != null) { + struct.put("identifier", identifier); + } struct.put("topic", original.topic()); struct.put("partition", original.kafkaPartition()); struct.put("offset", original.kafkaOffset()); @@ -154,6 +166,7 @@ private static SinkRecord failedRecord( struct.put("target_table", targetTable); } + struct.put("type", TYPE); return original.newRecord( original.topic(), original.kafkaPartition(), @@ -186,13 +199,13 @@ public static List serializedHeaders(SinkRecord original) { @SuppressWarnings("unchecked") public static SinkRecord mapToFailedRecord( - String targetTable, SinkRecord record, String location, Throwable error) { + String targetTable, SinkRecord record, String location, Throwable error, String identifier) { Map payload = (Map) record.value(); Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); Object keyBytes = bytes.get(KEY_BYTES); Object valueBytes = bytes.get(VALUE_BYTES); Object headers = bytes.get(HEADERS); Values values = new Values(keyBytes, valueBytes, headers); - return failedRecord(record, values, error, location, targetTable); + return failedRecord(record, values, error, location, targetTable, identifier); } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index b87b4478..0b65715d 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -124,7 +124,7 @@ SinkRecord handle(SinkRecord original, Throwable error, String location) { private static class AllExceptions extends ExceptionHandler { @Override SinkRecord handle(SinkRecord original, Throwable error, String location) { - return DeadLetterUtils.failedRecord(original, error, location); + return DeadLetterUtils.failedRecord(original, error, location, null); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index ec3fd0a5..c468dc3c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -101,6 +101,8 @@ public static class DeadLetterWriterForTable implements WriterForTable { private final Map writers; private final String deadLetterTableName; + private final String rowIdentifier; + DeadLetterWriterForTable( IcebergWriterFactory writerFactory, Map writers, @@ -110,6 +112,7 @@ public static class DeadLetterWriterForTable implements WriterForTable { Preconditions.checkNotNull( config.deadLetterTableName(), "Dead letter table name cannot be null"); this.deadLetterTableName = config.deadLetterTableName().toLowerCase(); + this.rowIdentifier = config.connectorName().toLowerCase().replace('-', '_'); } @SuppressWarnings("unchecked") @@ -137,6 +140,7 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl if (isFailed(record)) { Struct transformedStruct = (Struct) record.value(); transformedStruct.put("target_table", tableName); + transformedStruct.put("identifier", rowIdentifier); // not sure I should wrap this? // anything thrown here is a bug on our part, no? Someone has messed w/ the table? @@ -159,7 +163,7 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl } catch (Exception e) { SinkRecord newRecord = DeadLetterUtils.mapToFailedRecord( - tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e); + tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e, rowIdentifier); writers .computeIfAbsent( deadLetterTableName, @@ -176,7 +180,8 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl public void writeFailed( SinkRecord sample, String location, Throwable error, String targetTableName) { SinkRecord newRecord = - DeadLetterUtils.mapToFailedRecord(targetTableName, sample, location, error); + DeadLetterUtils.mapToFailedRecord( + targetTableName, sample, location, error, rowIdentifier); writers .computeIfAbsent( deadLetterTableName, diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index 747afa63..17382148 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -39,8 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -// TODO get the DLT options for partitioning from the config and look into how that works - public class IcebergWriterFactory { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); @@ -76,9 +74,11 @@ TableIdentifier id(String name) { private static class TableCreator { private final Catalog catalog; + private final IcebergSinkConfig config; - TableCreator(Catalog catalog) { + TableCreator(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; + this.config = config; } public Table createTable( @@ -88,12 +88,29 @@ public Table createTable( Map properties) { return catalog.createTable(identifier, schema, spec, properties); } + + public Schema schema(TableIdentifier identifier, SinkRecord sample) { + StructType structType; + if (sample.valueSchema() == null) { + structType = + SchemaUtils.inferIcebergType(sample.value(), config) + .orElseThrow(() -> new DataException("Unable to create table from empty object")) + .asStructType(); + } else { + structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); + } + + return new org.apache.iceberg.Schema(structType.fields()); + } } private class ErrorHandlingTableCreator extends TableCreator { - ErrorHandlingTableCreator(Catalog catalog) { - super(catalog); + private final TableIdentifier deadLetterTableId; + + ErrorHandlingTableCreator(Catalog catalog, IcebergSinkConfig config) { + super(catalog, config); + this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); } @Override @@ -102,20 +119,45 @@ public Table createTable( Schema schema, PartitionSpec spec, Map properties) { + Table table; - try { + if (identifier == deadLetterTableId) { table = catalog.createTable(identifier, schema, spec, properties); - } catch (IllegalArgumentException | ValidationException error) { - throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); + } else { + try { + table = catalog.createTable(identifier, schema, spec, properties); + } catch (IllegalArgumentException | ValidationException error) { + + // CommitStateUnknown exception. + // Look into NullPointer exceptions + // look into JsonProcessing exceptions inside of Iceberg + + throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); + } } return table; } + + @Override + public Schema schema(TableIdentifier identifier, SinkRecord sample) { + Schema schema; + if (identifier == deadLetterTableId) { + schema = super.schema(identifier, sample); + } else { + try { + schema = super.schema(identifier, sample); + } catch (IllegalArgumentException | ValidationException error) { + throw new DeadLetterUtils.DeadLetterException("CREATE_SCHEMA", error); + } + } + return schema; + } } public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; this.config = config; - this.tableCreator = new TableCreator(catalog); + this.tableCreator = new TableCreator(catalog, config); this.tableIdSupplier = new TableIdentifierSupplier(); if (config.autoCreateEnabled()) { @@ -123,7 +165,7 @@ public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { } else if (config.deadLetterTableEnabled()) { String deadLetterTableName = config.deadLetterTableName().toLowerCase(); shouldAutoCreate = (tableName) -> tableName.equals(deadLetterTableName); - this.tableCreator = new ErrorHandlingTableCreator(catalog); + this.tableCreator = new ErrorHandlingTableCreator(catalog, config); this.tableIdSupplier = new ErrorHandlingTableIdentifierSupplier(); } else { shouldAutoCreate = (unused) -> false; @@ -153,24 +195,11 @@ public RecordWriter createWriter( @VisibleForTesting Table autoCreateTable(String tableName, SinkRecord sample) { - StructType structType; - if (sample.valueSchema() == null) { - - // TODO thnk this one through and step through it - // NOT sure if we need to tackle this one. - - structType = - SchemaUtils.inferIcebergType(sample.value(), config) - .orElseThrow(() -> new DataException("Unable to create table from empty object")) - .asStructType(); - } else { - structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); - } - - org.apache.iceberg.Schema schema = new org.apache.iceberg.Schema(structType.fields()); TableIdentifier identifier = tableIdSupplier.id(tableName); + Schema schema = tableCreator.schema(identifier, sample); + List partitionBy = config.tableConfig(tableName).partitionBy(); PartitionSpec spec; @@ -192,7 +221,6 @@ Table autoCreateTable(String tableName, SinkRecord sample) { .run( notUsed -> { try { - // provided the identifier is valid, we can ignore this one. result.set(catalog.loadTable(identifier)); } catch (NoSuchTableException e) { result.set( diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 79a80bb2..07158adb 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -112,7 +112,7 @@ private SinkRecord errorTransformFailedRecord() { new SinkRecord( "topic", 0, null, null, null, ORIGINAL_BYTES, 100L, 1000L, TimestampType.CREATE_TIME); return DeadLetterUtils.failedRecord( - originalRecord, new IllegalArgumentException("test"), "location"); + originalRecord, new IllegalArgumentException("test"), "location", "test_identifier"); } @Test From edc75a5e7b81365f4eaeca41cea00ac8f5c6748d Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 9 Apr 2024 12:22:23 -0600 Subject: [PATCH 11/32] make converter/smt error converters configurable/user extensible --- .../transforms/DefaultExceptionHandler.java | 29 ++++++++++++ .../connect/transforms/ErrorTransform.java | 45 ++++++++++--------- .../transforms/TransformExceptionHandler.java | 25 +++++++++++ 3 files changed, 78 insertions(+), 21 deletions(-) create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java create mode 100644 kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java new file mode 100644 index 00000000..4a86e6f1 --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import org.apache.kafka.connect.sink.SinkRecord; + +public class DefaultExceptionHandler implements TransformExceptionHandler { + @Override + public SinkRecord handle(SinkRecord original, Throwable error, String location) { + return DeadLetterUtils.failedRecord(original, error, location, null); + } +} diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index 0b65715d..a1ef12b6 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -115,30 +115,20 @@ public boolean isFailed() { } } - private abstract static class ExceptionHandler { - SinkRecord handle(SinkRecord original, Throwable error, String location) { - throw new java.lang.IllegalStateException("handle not implemented"); - } - } - - private static class AllExceptions extends ExceptionHandler { - @Override - SinkRecord handle(SinkRecord original, Throwable error, String location) { - return DeadLetterUtils.failedRecord(original, error, location, null); - } - } - private static final String HEADERS = "headers"; private static final String KEY_CONVERTER = "key.converter"; private static final String VALUE_CONVERTER = "value.converter"; private static final String HEADER_CONVERTER = "header.converter"; + private static final String CONVERTER_ERROR_HANDLER = "error.converter"; + private static final String SMT_ERROR_HANDLER = "error.smt"; private static final String TRANSFORMATIONS = "smts"; private static final String KEY_FAILURE = "KEY_CONVERTER"; private static final String VALUE_FAILURE = "VALUE_CONVERTER"; private static final String HEADER_FAILURE = "HEADER_CONVERTER"; private static final String SMT_FAILURE = "SMT_FAILURE"; - private ExceptionHandler errorHandler; + private TransformExceptionHandler converterErrorHandler; + private TransformExceptionHandler smtErrorHandler; private List> smts; private Function keyConverter; private Function valueConverter; @@ -164,8 +154,19 @@ SinkRecord handle(SinkRecord original, Throwable error, String location) { "org.apache.kafka.connect.converters.ByteArrayConverter", ConfigDef.Importance.MEDIUM, "header.converter") + .define(TRANSFORMATIONS, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "smts") .define( - TRANSFORMATIONS, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "smts"); + CONVERTER_ERROR_HANDLER, + ConfigDef.Type.STRING, + "io.tabular.iceberg.connect.transforms.DefaultExceptionHandler", + ConfigDef.Importance.MEDIUM, + "Error handling class for converter errors") + .define( + SMT_ERROR_HANDLER, + ConfigDef.Type.STRING, + "io.tabular.iceberg.connect.transforms.DefaultExceptionHandler", + ConfigDef.Importance.MEDIUM, + "Error handling class for SMT errors"); @Override public SinkRecord apply(SinkRecord record) { @@ -188,7 +189,7 @@ public SinkRecord apply(SinkRecord record) { break; } } catch (Exception e) { - return errorHandler.handle(record, e, SMT_FAILURE); + return smtErrorHandler.handle(record, e, SMT_FAILURE); } } // SMT could filter out messages @@ -245,7 +246,6 @@ public void configure(Map props) { (HeaderConverter) loadClass("org.apache.kafka.connect.converters.ByteArrayConverter", loader)) { converter.configure(PropsParser.apply(props, HEADER_CONVERTER)); - headerConverter = converter; } catch (Exception e) { throw new TransformInitializationException( String.format( @@ -294,7 +294,10 @@ record -> { .collect(Collectors.toList()); } - errorHandler = new AllExceptions(); + converterErrorHandler = + (TransformExceptionHandler) loadClass(config.getString(CONVERTER_ERROR_HANDLER), loader); + smtErrorHandler = + (TransformExceptionHandler) loadClass(config.getString(SMT_ERROR_HANDLER), loader); } private Object loadClass(String name, ClassLoader loader) { @@ -320,18 +323,18 @@ private DeserializedRecord deserialize(SinkRecord record) { try { keyData = keyConverter.apply(record); } catch (Exception e) { - return new DeserializedRecord(errorHandler.handle(record, e, KEY_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e, KEY_FAILURE), true); } try { valueData = valueConverter.apply(record); } catch (Exception e) { - return new DeserializedRecord(errorHandler.handle(record, e, VALUE_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e, VALUE_FAILURE), true); } try { newHeaders = headerConverterFn.apply(record); } catch (Exception e) { - return new DeserializedRecord(errorHandler.handle(record, e, HEADER_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e, HEADER_FAILURE), true); } return new DeserializedRecord( diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java new file mode 100644 index 00000000..9b273638 --- /dev/null +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.transforms; + +import org.apache.kafka.connect.sink.SinkRecord; + +public interface TransformExceptionHandler { + SinkRecord handle(SinkRecord original, Throwable error, String location); +} From 8ee6840724b67f7ff9a770581920b79cc3f3934d Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 9 Apr 2024 15:57:27 -0600 Subject: [PATCH 12/32] introduce catalogApi to make IcebergWriterFactory testing easier --- .../iceberg/connect/channel/Worker.java | 9 +- .../iceberg/connect/data/CatalogApi.java | 131 ++++++++++++++++ .../connect/data/IcebergWriterFactory.java | 141 +++--------------- .../iceberg/connect/channel/WorkerTest.java | 78 +++++++++- .../data/IcebergWriterFactoryTest.java | 47 ++++++ 5 files changed, 276 insertions(+), 130 deletions(-) create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index c468dc3c..04b14e2b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -35,6 +35,7 @@ import io.tabular.iceberg.connect.events.EventType; import io.tabular.iceberg.connect.events.TableName; import io.tabular.iceberg.connect.events.TopicPartitionOffset; +import java.io.UncheckedIOException; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -68,7 +69,7 @@ public interface WriterForTable { void writeFailed(SinkRecord sample, String location, Throwable error, String targetTableName); } - private static class BaseWriterForTable implements WriterForTable { + public static class BaseWriterForTable implements WriterForTable { private final IcebergWriterFactory writerFactory; private final Map writers; @@ -160,10 +161,12 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl if (recordToWrite != null) { try { writer.write(recordToWrite); - } catch (Exception e) { + } catch (UncheckedIOException error) { + throw error; + } catch (Exception error) { SinkRecord newRecord = DeadLetterUtils.mapToFailedRecord( - tableName, record, ICEBERG_TRANSFORMATION_LOCATION, e, rowIdentifier); + tableName, record, ICEBERG_TRANSFORMATION_LOCATION, error, rowIdentifier); writers .computeIfAbsent( deadLetterTableName, diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java new file mode 100644 index 00000000..4f134dfa --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.types.Types; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.SinkRecord; + +public abstract class CatalogApi { + + private final Catalog catalog; + private final IcebergSinkConfig config; + + CatalogApi(Catalog catalog, IcebergSinkConfig config) { + this.catalog = catalog; + this.config = config; + } + + TableIdentifier tableId(String name) { + return TableIdentifier.parse(name); + } + + public final Table loadTable(TableIdentifier identifier) { + return catalog.loadTable(identifier); + } + + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + Map properties) { + return catalog.createTable(identifier, schema, spec, properties); + } + + public Schema schema(TableIdentifier identifier, SinkRecord sample) { + Types.StructType structType; + if (sample.valueSchema() == null) { + structType = + SchemaUtils.inferIcebergType(sample.value(), config) + .orElseThrow(() -> new DataException("Unable to create table from empty object")) + .asStructType(); + } else { + structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); + } + + return new org.apache.iceberg.Schema(structType.fields()); + } + + public static class ErrorHandlingCatalogApi extends CatalogApi { + + private final TableIdentifier deadLetterTableId; + private final Catalog catalog; + + ErrorHandlingCatalogApi(Catalog catalog, IcebergSinkConfig config) { + super(catalog, config); + this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); + this.catalog = catalog; + } + + @Override + TableIdentifier tableId(String name) { + TableIdentifier tableId; + try { + tableId = super.tableId(name); + } catch (Exception error) { + throw new DeadLetterUtils.DeadLetterException("TABLE_IDENTIFIER", error); + } + return tableId; + } + + @Override + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + Map properties) { + + Table table; + if (identifier == deadLetterTableId) { + table = catalog.createTable(identifier, schema, spec, properties); + } else { + try { + table = catalog.createTable(identifier, schema, spec, properties); + } catch (IllegalArgumentException | ValidationException error) { + throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); + } + } + return table; + } + + @Override + public Schema schema(TableIdentifier identifier, SinkRecord sample) { + Schema schema; + if (identifier == deadLetterTableId) { + schema = super.schema(identifier, sample); + } else { + try { + schema = super.schema(identifier, sample); + } catch (IllegalArgumentException | ValidationException error) { + throw new DeadLetterUtils.DeadLetterException("CREATE_SCHEMA", error); + } + } + return schema; + } + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index 17382148..f592995e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -19,9 +19,7 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import org.apache.iceberg.PartitionSpec; @@ -30,11 +28,8 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.Tasks; -import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,130 +38,25 @@ public class IcebergWriterFactory { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); - private final Catalog catalog; private final IcebergSinkConfig config; private final Predicate shouldAutoCreate; - private TableIdentifierSupplier tableIdSupplier; + private final CatalogApi catalogApi; - private TableCreator tableCreator; - - private static class TableIdentifierSupplier { - TableIdentifier id(String name) { - return TableIdentifier.parse(name); - } - } - - private static class ErrorHandlingTableIdentifierSupplier extends TableIdentifierSupplier { - @Override - TableIdentifier id(String name) { - TableIdentifier tableId; - try { - tableId = super.id(name); - } catch (Exception error) { - throw new DeadLetterUtils.DeadLetterException("TABLE_IDENTIFIER", error); - } - return tableId; - } - } - - private static class TableCreator { - - private final Catalog catalog; - private final IcebergSinkConfig config; - - TableCreator(Catalog catalog, IcebergSinkConfig config) { - this.catalog = catalog; - this.config = config; - } - - public Table createTable( - TableIdentifier identifier, - Schema schema, - PartitionSpec spec, - Map properties) { - return catalog.createTable(identifier, schema, spec, properties); - } - - public Schema schema(TableIdentifier identifier, SinkRecord sample) { - StructType structType; - if (sample.valueSchema() == null) { - structType = - SchemaUtils.inferIcebergType(sample.value(), config) - .orElseThrow(() -> new DataException("Unable to create table from empty object")) - .asStructType(); - } else { - structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); - } - - return new org.apache.iceberg.Schema(structType.fields()); - } - } - - private class ErrorHandlingTableCreator extends TableCreator { - - private final TableIdentifier deadLetterTableId; - - ErrorHandlingTableCreator(Catalog catalog, IcebergSinkConfig config) { - super(catalog, config); - this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); - } - - @Override - public Table createTable( - TableIdentifier identifier, - Schema schema, - PartitionSpec spec, - Map properties) { - - Table table; - if (identifier == deadLetterTableId) { - table = catalog.createTable(identifier, schema, spec, properties); - } else { - try { - table = catalog.createTable(identifier, schema, spec, properties); - } catch (IllegalArgumentException | ValidationException error) { - - // CommitStateUnknown exception. - // Look into NullPointer exceptions - // look into JsonProcessing exceptions inside of Iceberg - - throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); - } - } - return table; - } - - @Override - public Schema schema(TableIdentifier identifier, SinkRecord sample) { - Schema schema; - if (identifier == deadLetterTableId) { - schema = super.schema(identifier, sample); - } else { - try { - schema = super.schema(identifier, sample); - } catch (IllegalArgumentException | ValidationException error) { - throw new DeadLetterUtils.DeadLetterException("CREATE_SCHEMA", error); - } - } - return schema; - } + public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { + this(config, getCatalogApi(catalog, config)); } - public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { - this.catalog = catalog; + public IcebergWriterFactory(IcebergSinkConfig config, CatalogApi api) { this.config = config; - this.tableCreator = new TableCreator(catalog, config); - this.tableIdSupplier = new TableIdentifierSupplier(); + this.catalogApi = api; if (config.autoCreateEnabled()) { shouldAutoCreate = (unused) -> true; } else if (config.deadLetterTableEnabled()) { String deadLetterTableName = config.deadLetterTableName().toLowerCase(); shouldAutoCreate = (tableName) -> tableName.equals(deadLetterTableName); - this.tableCreator = new ErrorHandlingTableCreator(catalog, config); - this.tableIdSupplier = new ErrorHandlingTableIdentifierSupplier(); } else { shouldAutoCreate = (unused) -> false; } @@ -175,11 +65,10 @@ public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { public RecordWriter createWriter( String tableName, SinkRecord sample, boolean ignoreMissingTable) { - // this can fail. - TableIdentifier identifier = tableIdSupplier.id(tableName); + TableIdentifier identifier = catalogApi.tableId(tableName); Table table; try { - table = catalog.loadTable(identifier); + table = catalogApi.loadTable(identifier); } catch (NoSuchTableException nst) { if (shouldAutoCreate.test(tableName)) { table = autoCreateTable(tableName, sample); @@ -196,9 +85,9 @@ public RecordWriter createWriter( @VisibleForTesting Table autoCreateTable(String tableName, SinkRecord sample) { - TableIdentifier identifier = tableIdSupplier.id(tableName); + TableIdentifier identifier = catalogApi.tableId(tableName); - Schema schema = tableCreator.schema(identifier, sample); + Schema schema = catalogApi.schema(identifier, sample); List partitionBy = config.tableConfig(tableName).partitionBy(); @@ -221,13 +110,21 @@ Table autoCreateTable(String tableName, SinkRecord sample) { .run( notUsed -> { try { - result.set(catalog.loadTable(identifier)); + result.set(catalogApi.loadTable(identifier)); } catch (NoSuchTableException e) { result.set( - tableCreator.createTable( + catalogApi.createTable( identifier, schema, partitionSpec, config.autoCreateProps())); } }); return result.get(); } + + private static CatalogApi getCatalogApi(Catalog catalog, IcebergSinkConfig config) { + if (config.deadLetterTableEnabled()) { + return new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + } else { + return new CatalogApi(catalog, config) {}; + } + } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 07158adb..47e23323 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -34,6 +36,8 @@ import io.tabular.iceberg.connect.events.Event; import io.tabular.iceberg.connect.events.EventTestUtil; import io.tabular.iceberg.connect.events.EventType; +import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -183,11 +187,37 @@ private void workerTest(Map value) { assertThat(readyPayload.assignments().get(0).offset()).isEqualTo(1L); } + @Test + @DisplayName("BaseWriterForTable should create writers and pass records to them") + public void baseWriterForTable() { + when(config.deadLetterTableEnabled()).thenReturn(false); + + RecordingRecordWriter writer = new RecordingRecordWriter(false); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.BaseWriterForTable writerForTable = + new Worker.BaseWriterForTable(writerFactory, writerMap); + + SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, "test", 0); + writerForTable.write(TABLE_NAME, nullRecord, true); + + assertThat(writer.written.size()).isEqualTo(1); + assertThat(writer.written.get(0).value()).isEqualTo("test"); + assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(TABLE_NAME)); + } + @Test @DisplayName("DeadLetterWriterForTable should ignore null records") public void deadLetterWriterForTableNullRecords() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); RecordingRecordWriter writer = new RecordingRecordWriter(false); @@ -212,6 +242,7 @@ public void deadLetterWriterForTableNullRecords() { public void deadLetterWriterForTableSuccessErrorTransform() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); RecordingRecordWriter writer = new RecordingRecordWriter(false); @@ -243,6 +274,7 @@ public void deadLetterWriterForTableSuccessErrorTransform() { public void deadLetterWriterForTableFailureErrorTransform() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); RecordingRecordWriter writer = new RecordingRecordWriter(false); @@ -266,6 +298,8 @@ public void deadLetterWriterForTableFailureErrorTransform() { String resultBytesAsString = new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); + assertThat(resultValue.get("target_table")).isEqualTo(TABLE_NAME); + assertThat(resultValue.get("identifier")).isEqualTo("connector_name"); } @Test @@ -274,6 +308,8 @@ public void deadLetterWriterForTableFailureErrorTransform() { public void deadLetterWriterForTableConvertWriterFailures() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); + RecordingRecordWriter writer = new RecordingRecordWriter(true); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); @@ -296,6 +332,8 @@ public void deadLetterWriterForTableConvertWriterFailures() { String resultBytesAsString = new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); + assertThat(resultValue.get("target_table")).isEqualTo(TABLE_NAME); + assertThat(resultValue.get("identifier")).isEqualTo("connector_name"); // we successfully create the regular writer, but fail to write it, seeing both writers in the // Map assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(TABLE_NAME, DEAD_LETTER_TABLE_NAME)); @@ -307,10 +345,11 @@ public void deadLetterWriterForTableConvertWriterFailures() { public void deadLetterWriterThrowWriterCreation() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); when(writerFactory.createWriter( - TABLE_NAME, ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) .thenThrow(new IllegalArgumentException("writer factory throws")); Map writerMap = Maps.newHashMap(); @@ -329,19 +368,18 @@ public void deadLetterWriterThrowWriterCreation() { public void deadLetterWriterDeadLetterExceptionsAtCreation() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); RecordingRecordWriter writer = new RecordingRecordWriter(false); IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); when(writerFactory.createWriter( - ArgumentMatchers.eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) .thenThrow( new DeadLetterUtils.DeadLetterException( "test", new IllegalArgumentException("creation throw"))); when(writerFactory.createWriter( - ArgumentMatchers.eq(DEAD_LETTER_TABLE_NAME), - ArgumentMatchers.any(), - ArgumentMatchers.anyBoolean())) + eq(DEAD_LETTER_TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) .thenReturn(writer); Map writerMap = Maps.newHashMap(); @@ -362,4 +400,34 @@ public void deadLetterWriterDeadLetterExceptionsAtCreation() { // fail to create the original message, but do successfully create the dead letter table assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(DEAD_LETTER_TABLE_NAME)); } + + @Test + @DisplayName( + "DeadLetterWriterForTable should fail when UncheckedIOException is thrown on writing a record") + public void deadLetterWriterThrowWriterWriteUncheckedIO() { + // the writer may close/flush during writing due to schema changes being applied to the table + // which introduces the possibility of IOExceptions + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); + when(config.connectorName()).thenReturn("connector-name"); + RecordWriter writer = mock(IcebergWriter.class); + + doThrow(new UncheckedIOException(new IOException("test"))) + .when(writer) + .write(ArgumentMatchers.any()); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter( + ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) + .thenReturn(writer); + + Map writerMap = Maps.newHashMap(); + + Worker.DeadLetterWriterForTable writerForTable = + new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); + + assertThrows( + UncheckedIOException.class, + () -> writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true)); + } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java index b15e7394..e1f8864b 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java @@ -37,12 +37,16 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; public class IcebergWriterFactoryTest { + private static final String DEAD_LETTER_TABLE = "dlt.name"; + @ParameterizedTest @ValueSource(booleans = {true, false}) @SuppressWarnings("unchecked") @@ -83,4 +87,47 @@ public void testAutoCreateTable(boolean partitioned) { assertThat(specCaptor.getValue().isPartitioned()).isEqualTo(partitioned); assertThat(propsCaptor.getValue()).containsKey("test-prop"); } + + @Test + @DisplayName( + "createWriter should create a DeadLetterTable even when autocreatetables is not configured") + public void createWriterAutoCreatesDeadLetterTable() { + Catalog catalog = mock(Catalog.class); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.autoCreateEnabled()).thenReturn(false); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + } + + @Test + @DisplayName("createWriter should return DeadLetterException for invalid table names") + public void createWriterThrowsDeadLetterExceptions() { + // when configured for DeadLetterTable + } + + @Test + @DisplayName("createWriter should return DeadLetterException for select schema conversion errors") + public void createWriterThrowsOnSchemaConversionErrors() {} + + @Test + @DisplayName( + "createWriter should return underlying exception for schema conversion errors if table is dead letter table") + public void createWriterThrowsOnSchmeaConversionErrorsForDLT() {} + + @Test + @DisplayName("createWriter should return DeadLetterExceptions for select table creation errors") + public void createWriterThrowsTableCreationErrors() {} + + @Test + @DisplayName( + "createWriter should return underlying exception for table creation errors if table is dead letter table") + public void createWriterThrowsTableCreationErrorsDLT() {} + + @Test + @DisplayName("createWriter should apply partition spec from Configuration during table creation") + public void createWriterAppliesPartitionSpec() {} + + @Test + @DisplayName("createWriter should apply id column from Configuration during table creation") + public void createWriterAppliesIdColumn() {} } From de479d384437a5e80b6559f5bd243b244c9c8530 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 9 Apr 2024 17:11:59 -0600 Subject: [PATCH 13/32] catalogApi and test stubs --- .../iceberg/connect/channel/Worker.java | 4 - .../iceberg/connect/data/CatalogApi.java | 21 ++++ .../connect/data/IcebergWriterFactory.java | 20 +--- .../iceberg/connect/data/CatalogApiTest.java | 99 +++++++++++++++++++ .../data/IcebergWriterFactoryTest.java | 45 --------- 5 files changed, 121 insertions(+), 68 deletions(-) create mode 100644 kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 04b14e2b..100cbbec 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -142,10 +142,6 @@ public void write(String tableName, SinkRecord record, boolean ignoreMissingTabl Struct transformedStruct = (Struct) record.value(); transformedStruct.put("target_table", tableName); transformedStruct.put("identifier", rowIdentifier); - - // not sure I should wrap this? - // anything thrown here is a bug on our part, no? Someone has messed w/ the table? - // everything here should be valid at this point writer = writers.computeIfAbsent( deadLetterTableName, diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java index 4f134dfa..bbe54115 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java @@ -20,6 +20,7 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import java.util.List; import java.util.Map; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -30,8 +31,11 @@ import org.apache.iceberg.types.Types; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class CatalogApi { + private static final Logger LOG = LoggerFactory.getLogger(CatalogApi.class); private final Catalog catalog; private final IcebergSinkConfig config; @@ -49,6 +53,23 @@ public final Table loadTable(TableIdentifier identifier) { return catalog.loadTable(identifier); } + public final PartitionSpec partitionSpec(String tableName, Schema schema) { + List partitionBy = config.tableConfig(tableName).partitionBy(); + + PartitionSpec spec; + try { + spec = SchemaUtils.createPartitionSpec(schema, partitionBy); + } catch (Exception e) { + LOG.error( + "Unable to create partition spec {}, table {} will be unpartitioned", + partitionBy, + tableName, + e); + spec = PartitionSpec.unpartitioned(); + } + return spec; + } + public Table createTable( TableIdentifier identifier, Schema schema, diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index f592995e..29173336 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -19,7 +19,6 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; -import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import org.apache.iceberg.PartitionSpec; @@ -31,13 +30,9 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.util.Tasks; import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class IcebergWriterFactory { - private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); - private final IcebergSinkConfig config; private final Predicate shouldAutoCreate; @@ -89,21 +84,8 @@ Table autoCreateTable(String tableName, SinkRecord sample) { Schema schema = catalogApi.schema(identifier, sample); - List partitionBy = config.tableConfig(tableName).partitionBy(); - - PartitionSpec spec; - try { - spec = SchemaUtils.createPartitionSpec(schema, partitionBy); - } catch (Exception e) { - LOG.error( - "Unable to create partition spec {}, table {} will be unpartitioned", - partitionBy, - identifier, - e); - spec = PartitionSpec.unpartitioned(); - } + PartitionSpec partitionSpec = catalogApi.partitionSpec(tableName, schema); - PartitionSpec partitionSpec = spec; AtomicReference result = new AtomicReference<>(); Tasks.range(1) .retry(IcebergSinkConfig.CREATE_TABLE_RETRIES) diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java new file mode 100644 index 00000000..3b356aaf --- /dev/null +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import org.apache.iceberg.catalog.Catalog; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +public class CatalogApiTest { + + private static final String DEAD_LETTER_TABLE = "dlt.table"; + + @Test + @DisplayName("CatalogApi.tableId throw exceptions for invalid table names") + public void tableIdThrows() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + CatalogApi defaultCatalog = new CatalogApi(catalog, config) {}; + assertThrows(IllegalArgumentException.class, () -> defaultCatalog.tableId("")); + } + + @Test + @DisplayName( + "ErrorHandlingCatalogApi.tableId should throw DeadLetterExceptions for invalid table names") + public void errorTableIdThrows() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + CatalogApi errorHandlingCatalog = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + assertThrows(DeadLetterUtils.DeadLetterException.class, () -> errorHandlingCatalog.tableId("")); + } + + @Test + @DisplayName( + "ErrorHandlingCatalog constructor should throw if dead letter table has invalid name") + public void errorHandlingCatalogConstructorThrows() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(""); + assertThrows( + IllegalArgumentException.class, + () -> new CatalogApi.ErrorHandlingCatalogApi(catalog, config)); + } + + @Test + @DisplayName("CatalogAPI/Error.partitionSpec should apply the configured PartitionSpec") + public void catalogApiAppliesPartitionConfig() {} + + @Test + @DisplayName("CatalogAPI/Error.partitionSpec should create be unpartitioned if an error occurs") + public void catalogApiPartitionSpecUnpartitioned() {} + + @Test + @DisplayName("CatalogAPI.createTable should throw validation/illegal argument exceptions") + public void catalogCreateTableShouldThrow() {} + + @Test + @DisplayName( + "ErrorHandlingCatalogApi.createTable should wrap validation/illegal argument exceptions") + public void errorHandlingCatalogCreateTableShouldWrap() {} + + @Test + @DisplayName("ErrorHandlingCatalogApi.createTable should not wrap other exceptions") + public void errorHandlingCatalogCreateTableNotWrap() {} + + @Test + @DisplayName("CatalogAPI.schema should throw exceptions") + public void catalogApiSchemaThrowsExceptions() {} + + @Test + @DisplayName("ErrorHandlingCatalogAPI.schema should wrap validation/illegal argument exceptions") + public void errorHandlingCatalogSchemaShouldWrap() {} + + @Test + @DisplayName("ErrorHandlingCatalogAPI.schema should not wrap other exceptions") + public void errorHandlingCatalogNotWrap() {} +} diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java index e1f8864b..d1577813 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java @@ -37,8 +37,6 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; @@ -87,47 +85,4 @@ public void testAutoCreateTable(boolean partitioned) { assertThat(specCaptor.getValue().isPartitioned()).isEqualTo(partitioned); assertThat(propsCaptor.getValue()).containsKey("test-prop"); } - - @Test - @DisplayName( - "createWriter should create a DeadLetterTable even when autocreatetables is not configured") - public void createWriterAutoCreatesDeadLetterTable() { - Catalog catalog = mock(Catalog.class); - - IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.autoCreateEnabled()).thenReturn(false); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); - } - - @Test - @DisplayName("createWriter should return DeadLetterException for invalid table names") - public void createWriterThrowsDeadLetterExceptions() { - // when configured for DeadLetterTable - } - - @Test - @DisplayName("createWriter should return DeadLetterException for select schema conversion errors") - public void createWriterThrowsOnSchemaConversionErrors() {} - - @Test - @DisplayName( - "createWriter should return underlying exception for schema conversion errors if table is dead letter table") - public void createWriterThrowsOnSchmeaConversionErrorsForDLT() {} - - @Test - @DisplayName("createWriter should return DeadLetterExceptions for select table creation errors") - public void createWriterThrowsTableCreationErrors() {} - - @Test - @DisplayName( - "createWriter should return underlying exception for table creation errors if table is dead letter table") - public void createWriterThrowsTableCreationErrorsDLT() {} - - @Test - @DisplayName("createWriter should apply partition spec from Configuration during table creation") - public void createWriterAppliesPartitionSpec() {} - - @Test - @DisplayName("createWriter should apply id column from Configuration during table creation") - public void createWriterAppliesIdColumn() {} } From b78a68d793ed05ee4c095b9cbe8c841431b151c4 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 10 Apr 2024 09:48:09 -0600 Subject: [PATCH 14/32] finished tests --- .../iceberg/connect/channel/Worker.java | 2 + .../iceberg/connect/data/CatalogApi.java | 20 +- .../iceberg/connect/data/CatalogApiTest.java | 211 ++++++++++++++++-- 3 files changed, 215 insertions(+), 18 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 100cbbec..883b4875 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -112,6 +112,8 @@ public static class DeadLetterWriterForTable implements WriterForTable { this.writers = writers; Preconditions.checkNotNull( config.deadLetterTableName(), "Dead letter table name cannot be null"); + Preconditions.checkArgument( + config.deadLetterTableName().isEmpty(), "Dead letter table name cannot be empty"); this.deadLetterTableName = config.deadLetterTableName().toLowerCase(); this.rowIdentifier = config.connectorName().toLowerCase().replace('-', '_'); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java index bbe54115..1658ac4a 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java @@ -22,12 +22,14 @@ import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.util.List; import java.util.Map; +import java.util.function.BiFunction; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.types.Types; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; @@ -97,10 +99,24 @@ public static class ErrorHandlingCatalogApi extends CatalogApi { private final TableIdentifier deadLetterTableId; private final Catalog catalog; + private final BiFunction schemaFactory; + ErrorHandlingCatalogApi(Catalog catalog, IcebergSinkConfig config) { super(catalog, config); this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); this.catalog = catalog; + this.schemaFactory = super::schema; + } + + @VisibleForTesting + ErrorHandlingCatalogApi( + Catalog catalog, + IcebergSinkConfig config, + BiFunction schemaFactory) { + super(catalog, config); + this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); + this.catalog = catalog; + this.schemaFactory = schemaFactory; } @Override @@ -138,10 +154,10 @@ public Table createTable( public Schema schema(TableIdentifier identifier, SinkRecord sample) { Schema schema; if (identifier == deadLetterTableId) { - schema = super.schema(identifier, sample); + schema = this.schemaFactory.apply(identifier, sample); } else { try { - schema = super.schema(identifier, sample); + schema = this.schemaFactory.apply(identifier, sample); } catch (IllegalArgumentException | ValidationException error) { throw new DeadLetterUtils.DeadLetterException("CREATE_SCHEMA", error); } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java index 3b356aaf..738a4cb9 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java @@ -18,19 +18,47 @@ */ package io.tabular.iceberg.connect.data; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.TableSinkConfig; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import java.util.function.BiFunction; +import java.util.regex.Pattern; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentMatchers; public class CatalogApiTest { private static final String DEAD_LETTER_TABLE = "dlt.table"; + private static final TableIdentifier DEAD_LETTER_TABLE_ID = + TableIdentifier.parse(DEAD_LETTER_TABLE); + + private static final org.apache.kafka.connect.data.Schema SCHEMA = + SchemaBuilder.struct() + .field("a", Schema.STRING_SCHEMA) + .field("b", Schema.STRING_SCHEMA) + .build(); + + private SinkRecord sinkRecord() { + Struct struct = new Struct(SCHEMA); + struct.put("a", "a"); + struct.put("b", "b"); + return new SinkRecord("some-topic", 0, null, null, SCHEMA, struct, 100L); + } @Test @DisplayName("CatalogApi.tableId throw exceptions for invalid table names") @@ -64,36 +92,187 @@ public void errorHandlingCatalogConstructorThrows() { () -> new CatalogApi.ErrorHandlingCatalogApi(catalog, config)); } + @Test + @DisplayName("ErrorHandlingCatalogAPI.schema should wrap validation/illegal argument exceptions") + public void errorHandlingCatalogSchemaShouldWrap() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + + BiFunction illegalArgFn = + (a, b) -> { + throw new IllegalArgumentException("test"); + }; + + BiFunction validationExceptionFn = + (a, b) -> { + throw new ValidationException("test"); + }; + + CatalogApi catalogApiIllegalArg = + new CatalogApi.ErrorHandlingCatalogApi(catalog, config, illegalArgFn); + CatalogApi catalogApiValidationExp = + new CatalogApi.ErrorHandlingCatalogApi(catalog, config, validationExceptionFn); + + assertThrows( + DeadLetterUtils.DeadLetterException.class, + () -> catalogApiIllegalArg.schema(DEAD_LETTER_TABLE_ID, sinkRecord())); + assertThrows( + DeadLetterUtils.DeadLetterException.class, + () -> catalogApiValidationExp.schema(DEAD_LETTER_TABLE_ID, sinkRecord())); + } + @Test @DisplayName("CatalogAPI/Error.partitionSpec should apply the configured PartitionSpec") - public void catalogApiAppliesPartitionConfig() {} + public void catalogApiAppliesPartitionConfig() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + TableSinkConfig tableConfig = + new TableSinkConfig( + Pattern.compile(".*123", Pattern.DOTALL), + Lists.newArrayList(), + Lists.newArrayList("a"), + null); + + when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); + CatalogApi catalogApi = new CatalogApi(catalog, config) {}; + CatalogApi errorApi = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + + org.apache.iceberg.Schema schema = catalogApi.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); + assertThat(catalogApi.partitionSpec(DEAD_LETTER_TABLE, schema).isPartitioned()).isTrue(); + assertThat(errorApi.partitionSpec(DEAD_LETTER_TABLE, schema).isPartitioned()).isTrue(); + } @Test @DisplayName("CatalogAPI/Error.partitionSpec should create be unpartitioned if an error occurs") - public void catalogApiPartitionSpecUnpartitioned() {} + public void catalogApiPartitionSpecUnpartitioned() { + Catalog catalog = mock(Catalog.class); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + // partition on a field that does not exist + TableSinkConfig tableConfig = + new TableSinkConfig( + Pattern.compile(".*123", Pattern.DOTALL), + Lists.newArrayList(), + Lists.newArrayList("does_not_exist"), + null); + + when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); + CatalogApi catalogApi = new CatalogApi(catalog, config) {}; + CatalogApi errorApi = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + + org.apache.iceberg.Schema schema = catalogApi.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); + assertThat(catalogApi.partitionSpec(DEAD_LETTER_TABLE, schema).isUnpartitioned()).isTrue(); + assertThat(errorApi.partitionSpec(DEAD_LETTER_TABLE, schema).isUnpartitioned()).isTrue(); + } @Test @DisplayName("CatalogAPI.createTable should throw validation/illegal argument exceptions") - public void catalogCreateTableShouldThrow() {} + public void catalogCreateTableShouldThrow() { + Catalog catalogValidationException = mock(Catalog.class); + when(catalogValidationException.createTable( + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any())) + .thenThrow(new ValidationException("test")); + + Catalog catalogIllegalArgException = mock(Catalog.class); + when(catalogIllegalArgException.createTable( + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any())) + .thenThrow(new IllegalArgumentException("test")); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + TableSinkConfig tableConfig = + new TableSinkConfig( + Pattern.compile(".*123", Pattern.DOTALL), + Lists.newArrayList(), + Lists.newArrayList("a"), + null); + + when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); + CatalogApi catalogApiValidation = new CatalogApi(catalogValidationException, config) {}; + CatalogApi catalogApiIllegal = new CatalogApi(catalogIllegalArgException, config) {}; + + org.apache.iceberg.Schema schema = + catalogApiValidation.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); + + assertThrows( + ValidationException.class, + () -> + catalogApiValidation.createTable( + DEAD_LETTER_TABLE_ID, + schema, + catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), + Maps.newHashMap())); + assertThrows( + IllegalArgumentException.class, + () -> + catalogApiIllegal.createTable( + DEAD_LETTER_TABLE_ID, + schema, + catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), + Maps.newHashMap())); + } @Test @DisplayName( "ErrorHandlingCatalogApi.createTable should wrap validation/illegal argument exceptions") - public void errorHandlingCatalogCreateTableShouldWrap() {} + public void errorHandlingCatalogCreateTableShouldWrap() { + Catalog catalogValidationException = mock(Catalog.class); + when(catalogValidationException.createTable( + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any())) + .thenThrow(new ValidationException("test")); - @Test - @DisplayName("ErrorHandlingCatalogApi.createTable should not wrap other exceptions") - public void errorHandlingCatalogCreateTableNotWrap() {} + Catalog catalogIllegalArgException = mock(Catalog.class); + when(catalogIllegalArgException.createTable( + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any(), + ArgumentMatchers.any())) + .thenThrow(new IllegalArgumentException("test")); - @Test - @DisplayName("CatalogAPI.schema should throw exceptions") - public void catalogApiSchemaThrowsExceptions() {} + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); + TableSinkConfig tableConfig = + new TableSinkConfig( + Pattern.compile(".*123", Pattern.DOTALL), + Lists.newArrayList(), + Lists.newArrayList("a"), + null); - @Test - @DisplayName("ErrorHandlingCatalogAPI.schema should wrap validation/illegal argument exceptions") - public void errorHandlingCatalogSchemaShouldWrap() {} + when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); + CatalogApi catalogApiValidation = + new CatalogApi.ErrorHandlingCatalogApi(catalogValidationException, config) {}; + CatalogApi catalogApiIllegal = + new CatalogApi.ErrorHandlingCatalogApi(catalogIllegalArgException, config) {}; - @Test - @DisplayName("ErrorHandlingCatalogAPI.schema should not wrap other exceptions") - public void errorHandlingCatalogNotWrap() {} + org.apache.iceberg.Schema schema = + catalogApiValidation.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); + + assertThrows( + DeadLetterUtils.DeadLetterException.class, + () -> + catalogApiValidation.createTable( + DEAD_LETTER_TABLE_ID, + schema, + catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), + Maps.newHashMap())); + assertThrows( + DeadLetterUtils.DeadLetterException.class, + () -> + catalogApiIllegal.createTable( + DEAD_LETTER_TABLE_ID, + schema, + catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), + Maps.newHashMap())); + } } From ac5d6a56660b183ee048531f86e37b40d9346e6f Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 10 Apr 2024 10:50:51 -0600 Subject: [PATCH 15/32] negate --- .../main/java/io/tabular/iceberg/connect/channel/Worker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 883b4875..9833b7dc 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -113,7 +113,7 @@ public static class DeadLetterWriterForTable implements WriterForTable { Preconditions.checkNotNull( config.deadLetterTableName(), "Dead letter table name cannot be null"); Preconditions.checkArgument( - config.deadLetterTableName().isEmpty(), "Dead letter table name cannot be empty"); + !config.deadLetterTableName().isEmpty(), "Dead letter table name cannot be empty"); this.deadLetterTableName = config.deadLetterTableName().toLowerCase(); this.rowIdentifier = config.connectorName().toLowerCase().replace('-', '_'); } From 50b300bea3b2f09e2d44173f769eaacb1fa6a0e2 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 26 Apr 2024 16:39:49 -0700 Subject: [PATCH 16/32] dead-letter-table - substantiall reworks the PR - error transform and connector are connected via FailedRecordFactory - users can plug in their own schema shape for the failed records - users can dispatch to whatever dead letter table they want - bunch of classes moved around - pluggable WriteExceptionHandler introducered to catch failures - lots of code updated with custom WriteExceptions meant to be caught by the WriteExceptionHandler - works with or without the ErrorTransform in play --- README.md | 17 + .../connect/deadletter/DeadLetterUtils.java | 173 +++------ .../DefaultFailedRecordFactory.java | 161 +++++++++ .../deadletter/FailedRecordFactory.java | 40 +++ .../transforms/DefaultExceptionHandler.java | 34 +- .../connect/transforms/ErrorTransform.java | 52 +-- .../transforms/TransformExceptionHandler.java | 6 + .../transforms/ErrorTransformTest.java | 187 +++++----- .../iceberg/connect/IcebergSinkConfig.java | 36 +- .../iceberg/connect/channel/Worker.java | 328 +----------------- .../iceberg/connect/data/BaseCatalogApi.java | 138 ++++++++ .../iceberg/connect/data/CatalogApi.java | 139 +------- .../data/DefaultWriteExceptionHandler.java | 94 +++++ .../iceberg/connect/data/IcebergWriter.java | 9 +- .../connect/data/IcebergWriterFactory.java | 26 +- .../iceberg/connect/data/RecordConverter.java | 11 +- .../iceberg/connect/data/RecordRouter.java | 194 +++++++++++ .../iceberg/connect/data/SchemaUtils.java | 10 +- .../iceberg/connect/data/WriteException.java | 103 ++++++ .../connect/data/WriteExceptionHandler.java | 60 ++++ .../iceberg/connect/data/WriterManager.java | 58 ++++ .../iceberg/connect/channel/WorkerTest.java | 284 --------------- ...ogApiTest.java => BaseCatalogApiTest.java} | 128 ++----- .../data/IcebergWriterFactoryTest.java | 2 - 24 files changed, 1176 insertions(+), 1114 deletions(-) create mode 100644 kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java create mode 100644 kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriterManager.java rename kafka-connect/src/test/java/io/tabular/iceberg/connect/data/{CatalogApiTest.java => BaseCatalogApiTest.java} (54%) diff --git a/README.md b/README.md index 7ce5c79e..c5d94eb5 100644 --- a/README.md +++ b/README.md @@ -20,6 +20,23 @@ The zip archive will be found under `./kafka-connect-runtime/build/distributions # Configuration +iceberg.deadletter.enabled + +// this is now on the transform somewhere +// and MUST be supplied +// and is validated that FailedRecordFactory.schema provides it. +iceberg.deadletter.table_route_field // sure +// but also needs to be on the connector I think... I don't think the connector sees the transform settings <- LOOK INTO THIS + + +// iceberg.deadletter.table + +// iceberg.table.\
.partition-by <- we can add a partitioning scheme to it +// probably topic + +// connector that is handling 10 topics for customer A going to Catalog A + + | Property | Description | |--------------------------------------------|------------------------------------------------------------------------------------------------------------------| | iceberg.tables | Comma-separated list of destination tables | diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index ac24f92b..30f0039c 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -21,7 +21,6 @@ import java.io.PrintWriter; import java.io.StringWriter; import java.util.List; -import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -54,38 +53,19 @@ private DeadLetterUtils() { throw new IllegalStateException("Should not be initialialized"); } - public static final String TYPE = "dlt"; - public static final String KEY_BYTES = "key"; - public static final String VALUE_BYTES = "value"; - public static final String PAYLOAD_KEY = "transformed"; - public static final String ORIGINAL_BYTES_KEY = "original"; - private static final String HEADERS = "headers"; + public static final String KEY_HEADER = "t_original_key"; + public static final String VALUE_HEADER = "t_original_value"; + + public static final String HEADERS_HEADER = "t_original_headers"; public static final Schema HEADER_ELEMENT_SCHEMA = SchemaBuilder.struct() .field("key", Schema.STRING_SCHEMA) .field("value", Schema.OPTIONAL_BYTES_SCHEMA) .optional() .build(); + public static final Schema HEADER_SCHEMA = SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); - public static final Schema FAILED_SCHEMA = - SchemaBuilder.struct() - .name("failed_message") - .parameter("isFailed", "true") - .field("identifier", Schema.OPTIONAL_STRING_SCHEMA) - .field("topic", Schema.STRING_SCHEMA) - .field("partition", Schema.INT32_SCHEMA) - .field("offset", Schema.INT64_SCHEMA) - .field("location", Schema.STRING_SCHEMA) - .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) - .field("exception", Schema.OPTIONAL_STRING_SCHEMA) - .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) - .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) - .field(HEADERS, HEADER_SCHEMA) - .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) - .field("type", Schema.STRING_SCHEMA) - .schema(); public static String stackTrace(Throwable error) { StringWriter sw = new StringWriter(); @@ -94,88 +74,32 @@ public static String stackTrace(Throwable error) { return sw.toString(); } - public static class Values { - // expect byte[] - private final Object keyBytes; - // expect byte[] - private final Object valueBytes; - // expect List - private final Object headers; - - public Values(Object keyBytes, Object valueBytes, Object headers) { - this.keyBytes = keyBytes; - this.valueBytes = valueBytes; - this.headers = headers; - } - - public Object getKeyBytes() { - return keyBytes; - } - - public Object getValueBytes() { - return valueBytes; - } - - public Object getHeaders() { - return headers; - } - } - - public static SinkRecord failedRecord( - SinkRecord original, Throwable error, String location, String identifier) { - List headers = null; - if (!original.headers().isEmpty()) { - headers = DeadLetterUtils.serializedHeaders(original); - } - Values values = new Values(original.key(), original.value(), headers); - return failedRecord(original, values, error, location, null, identifier); - } - - private static SinkRecord failedRecord( - SinkRecord original, - Values values, - Throwable error, - String location, - String targetTable, - String identifier) { - - Struct struct = new Struct(FAILED_SCHEMA); - if (identifier != null) { - struct.put("identifier", identifier); - } - struct.put("topic", original.topic()); - struct.put("partition", original.kafkaPartition()); - struct.put("offset", original.kafkaOffset()); - struct.put("timestamp", original.timestamp()); - struct.put("location", location); - struct.put("exception", error.toString()); - String stack = stackTrace(error); - if (!stack.isEmpty()) { - struct.put("stack_trace", stackTrace(error)); - } - if (values.getKeyBytes() != null) { - struct.put("key_bytes", values.getKeyBytes()); - } - if (values.getValueBytes() != null) { - struct.put("value_bytes", values.getValueBytes()); - } - if (values.getHeaders() != null) { - struct.put(HEADERS, values.getHeaders()); - } - if (targetTable != null) { - struct.put("target_table", targetTable); - } - - struct.put("type", TYPE); - return original.newRecord( - original.topic(), - original.kafkaPartition(), - null, - null, - FAILED_SCHEMA, - struct, - original.timestamp()); - } + // public static class Values { + // // expect byte[] + // private final Object keyBytes; + // // expect byte[] + // private final Object valueBytes; + // // expect List + // private final Object headers; + // + // public Values(Object keyBytes, Object valueBytes, Object headers) { + // this.keyBytes = keyBytes; + // this.valueBytes = valueBytes; + // this.headers = headers; + // } + // + // public Object getKeyBytes() { + // return keyBytes; + // } + // + // public Object getValueBytes() { + // return valueBytes; + // } + // + // public Object getHeaders() { + // return headers; + // } + // } /** * No way to get back the original Kafka header bytes. We instead have an array with elements of @@ -197,15 +121,30 @@ public static List serializedHeaders(SinkRecord original) { return headers; } - @SuppressWarnings("unchecked") - public static SinkRecord mapToFailedRecord( - String targetTable, SinkRecord record, String location, Throwable error, String identifier) { - Map payload = (Map) record.value(); - Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); - Object keyBytes = bytes.get(KEY_BYTES); - Object valueBytes = bytes.get(VALUE_BYTES); - Object headers = bytes.get(HEADERS); - Values values = new Values(keyBytes, valueBytes, headers); - return failedRecord(record, values, error, location, targetTable, identifier); + // @SuppressWarnings("unchecked") + // public static SinkRecord mapToFailedRecord( + // String targetTable, SinkRecord record, String location, Throwable error, String + // identifier) { + // Map payload = (Map) record.value(); + // Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); + // Object keyBytes = bytes.get(KEY_BYTES); + // Object valueBytes = bytes.get(VALUE_BYTES); + // Object headers = bytes.get(HEADERS); + // Values values = new Values(keyBytes, valueBytes, headers); + // return failedRecord(record, values, error, location, targetTable, identifier); + // } + + public static Object loadClass(String name, ClassLoader loader) { + if (name == null || name.isEmpty()) { + throw new IllegalArgumentException("cannot initialize empty class"); + } + Object obj; + try { + Class clazz = Class.forName(name, true, loader); + obj = clazz.getDeclaredConstructor().newInstance(); + } catch (Exception e) { + throw new RuntimeException(String.format("could not initialize class %s", name), e); + } + return obj; } } diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java new file mode 100644 index 00000000..d76ed7c9 --- /dev/null +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.deadletter; + +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +public class DefaultFailedRecordFactory implements FailedRecordFactory { + + private static final String DEAD_LETTER_TABLE_NAME_PROP = "table_name"; + private static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + DEAD_LETTER_TABLE_NAME_PROP, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + "dead letter table name namespace.table"); + + private static final String HEADERS = "headers"; + private Schema schema; + + private String deadLetterTableName; + + @Override + public Schema schema(String context) { + return schema; + } + + @Override + public SinkRecord recordFromSmt(SinkRecord original, Throwable error, String context) { + Struct struct = new Struct(schema); + addCommon(struct, original, error, context); + + if (original.key() != null) { + struct.put("key_bytes", original.key()); + } + if (original.value() != null) { + struct.put("value_bytes", original.value()); + } + if (!original.headers().isEmpty()) { + struct.put(HEADERS, DeadLetterUtils.serializedHeaders(original)); + } + + return original.newRecord( + original.topic(), + original.kafkaPartition(), + null, + null, + schema, + struct, + original.timestamp()); + } + + @Override + public SinkRecord recordFromConnector(SinkRecord record, Throwable error, String context) { + + Struct struct = new Struct(schema); + addCommon(struct, record, error, context); + + Headers headers = record.headers(); + Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); + Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); + Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); + + if (keyHeader != null) { + struct.put("key_bytes", keyHeader.value()); + } + if (valueHeader != null) { + struct.put("value_bytes", valueHeader.value()); + } + if (serializedHeader != null) { + struct.put(HEADERS, serializedHeader.value()); + } + + return record.newRecord( + record.topic(), record.kafkaPartition(), null, null, schema, struct, record.timestamp()); + } + + @Override + public boolean isFailedTransformRecord(SinkRecord record) { + if (record != null && record.valueSchema() != null) { + Map parameters = record.valueSchema().parameters(); + if (parameters != null) { + String isFailed = parameters.get("transform_failed"); + if (isFailed != null) { + return isFailed.equals("true"); + } + } + } + return false; + } + + @Override + public String tableName(SinkRecord record) { + return deadLetterTableName; + } + + @Override + public void configure(Map props) { + SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + deadLetterTableName = config.getString(DEAD_LETTER_TABLE_NAME_PROP); + if (deadLetterTableName == null) { + throw new IllegalArgumentException("Dead letter table name cannot be null"); + } + schema = + SchemaBuilder.struct() + .name("failed_message") + .parameter("transform_failed", "true") + .field("topic", Schema.STRING_SCHEMA) + .field("partition", Schema.INT32_SCHEMA) + .field("offset", Schema.INT64_SCHEMA) + .field("timestamp", Schema.OPTIONAL_INT64_SCHEMA) + .field("exception", Schema.OPTIONAL_STRING_SCHEMA) + .field("stack_trace", Schema.OPTIONAL_STRING_SCHEMA) + .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) + .field(HEADERS, DeadLetterUtils.HEADER_SCHEMA) + .field("context", Schema.OPTIONAL_STRING_SCHEMA) + .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) + .schema(); + } + + private void addCommon(Struct struct, SinkRecord record, Throwable error, String context) { + struct.put("topic", record.topic()); + struct.put("partition", record.kafkaPartition()); + struct.put("offset", record.kafkaOffset()); + struct.put("timestamp", record.timestamp()); + struct.put("exception", error.toString()); + String stack = DeadLetterUtils.stackTrace(error); + if (!stack.isEmpty()) { + struct.put("stack_trace", stack); + } + if (context != null) { + struct.put("context", context); + } + } +} diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java new file mode 100644 index 00000000..d0ff167b --- /dev/null +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.deadletter; + +import java.util.Map; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.sink.SinkRecord; + +public interface FailedRecordFactory { + Schema schema(String context); + + // how to take SMT record (which FYI is all ByteArrays) and turn it into some form of FailedRecord + SinkRecord recordFromSmt(SinkRecord original, Throwable error, String context); + + // here is where it starts getting awkward + // where in the original are the byte arrays. + SinkRecord recordFromConnector(SinkRecord record, Throwable error, String context); + + boolean isFailedTransformRecord(SinkRecord record); + + String tableName(SinkRecord record); + + void configure(Map props); +} diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java index 4a86e6f1..d3319023 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java @@ -19,11 +19,43 @@ package io.tabular.iceberg.connect.transforms; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.transforms.util.SimpleConfig; public class DefaultExceptionHandler implements TransformExceptionHandler { + + private static final String FAILED_RECORD_FACTORY_PROP = "failed_record_factory"; + private static final ConfigDef CONFIG_DEF = + new ConfigDef() + .define( + FAILED_RECORD_FACTORY_PROP, + ConfigDef.Type.STRING, + "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory", + ConfigDef.Importance.MEDIUM, + "class name for failed record conversion"); + + private FailedRecordFactory recordFactory; + @Override public SinkRecord handle(SinkRecord original, Throwable error, String location) { - return DeadLetterUtils.failedRecord(original, error, location, null); + return recordFactory.recordFromSmt(original, error, location); + } + + @Override + public void configure(Map props) { + SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + ClassLoader loader = this.getClass().getClassLoader(); + this.recordFactory = + (FailedRecordFactory) + DeadLetterUtils.loadClass(config.getString(FAILED_RECORD_FACTORY_PROP), loader); + recordFactory.configure(props); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index a1ef12b6..bb61bffe 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -26,10 +26,12 @@ import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; @@ -115,7 +117,6 @@ public boolean isFailed() { } } - private static final String HEADERS = "headers"; private static final String KEY_CONVERTER = "key.converter"; private static final String VALUE_CONVERTER = "value.converter"; private static final String HEADER_CONVERTER = "header.converter"; @@ -126,6 +127,7 @@ public boolean isFailed() { private static final String VALUE_FAILURE = "VALUE_CONVERTER"; private static final String HEADER_FAILURE = "HEADER_CONVERTER"; private static final String SMT_FAILURE = "SMT_FAILURE"; + private static final Schema OPTIONAL_BYTES_SCHEMA = SchemaBuilder.OPTIONAL_BYTES_SCHEMA; private TransformExceptionHandler converterErrorHandler; private TransformExceptionHandler smtErrorHandler; @@ -296,8 +298,10 @@ record -> { converterErrorHandler = (TransformExceptionHandler) loadClass(config.getString(CONVERTER_ERROR_HANDLER), loader); + converterErrorHandler.configure(props); smtErrorHandler = (TransformExceptionHandler) loadClass(config.getString(SMT_ERROR_HANDLER), loader); + smtErrorHandler.configure(props); } private Object loadClass(String name, ClassLoader loader) { @@ -351,33 +355,29 @@ private DeserializedRecord deserialize(SinkRecord record) { } private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { - Map bytes = Maps.newHashMap(); - - if (original.key() != null) { - bytes.put(DeadLetterUtils.KEY_BYTES, original.key()); + if (!original.headers().isEmpty()) { + List serializedHeaders = DeadLetterUtils.serializedHeaders(original); + transformed + .headers() + .add( + DeadLetterUtils.HEADERS_HEADER, + new SchemaAndValue(DeadLetterUtils.HEADER_SCHEMA, serializedHeaders)); } - if (original.value() == null) { - throw new IllegalStateException("newRecord called with null value for record.value"); + if (original.key() != null) { + transformed + .headers() + .add( + DeadLetterUtils.KEY_HEADER, + new SchemaAndValue(OPTIONAL_BYTES_SCHEMA, original.key())); } - - if (!original.headers().isEmpty()) { - bytes.put(HEADERS, DeadLetterUtils.serializedHeaders(original)); + if (original.value() != null) { + transformed + .headers() + .add( + DeadLetterUtils.VALUE_HEADER, + new SchemaAndValue(OPTIONAL_BYTES_SCHEMA, original.value())); } - bytes.put(DeadLetterUtils.VALUE_BYTES, original.value()); - - Map result = Maps.newHashMap(); - result.put(DeadLetterUtils.PAYLOAD_KEY, transformed); - result.put(DeadLetterUtils.ORIGINAL_BYTES_KEY, bytes); - - return transformed.newRecord( - transformed.topic(), - transformed.kafkaPartition(), - null, - null, - null, - result, - transformed.timestamp(), - transformed.headers()); + return transformed; } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java index 9b273638..ee5bf881 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java @@ -18,8 +18,14 @@ */ package io.tabular.iceberg.connect.transforms; +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.sink.SinkRecord; public interface TransformExceptionHandler { SinkRecord handle(SinkRecord original, Throwable error, String location); + + void configure(Map props); + + ConfigDef config(); } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java index 0b81524e..e284aba3 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -22,6 +22,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -30,8 +31,10 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.jupiter.api.DisplayName; @@ -52,10 +55,14 @@ public class ErrorTransformTest { private static final String JSON_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; private static final String STRING_CONVERTER = "org.apache.kafka.connect.storage.StringConverter"; + private static final String DEAD_LETTER_TABLE_NAME = "dead_letter.table"; + + private final FailedRecordFactory failedRecordFactory = getFailedRecordFactory(); + private Headers stringAsByteHeaders() { Headers headers = new ConnectHeaders(); headers.add( - "h1", new SchemaAndValue(Schema.BYTES_SCHEMA, "h1".getBytes(StandardCharsets.UTF_8))); + "h1k", new SchemaAndValue(Schema.BYTES_SCHEMA, "h1v".getBytes(StandardCharsets.UTF_8))); return headers; } @@ -76,9 +83,24 @@ private SinkRecord createRecord(String key, String value, Headers headers) { headers); } + private FailedRecordFactory getFailedRecordFactory() { + FailedRecordFactory factory = + (FailedRecordFactory) + DeadLetterUtils.loadClass( + "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory", + this.getClass().getClassLoader()); + factory.configure(ImmutableMap.of("table_name", DEAD_LETTER_TABLE_NAME)); + return factory; + } + + // private assertHeaders(Headers headers, String valueBytes, String keyBytes, List + // originalHeaders) { + // headers.lastWithName("t_") + // } + @Test @DisplayName( - "It should deserialize using the supplied converters into the custom SinkRecord shape with original/transformed result") + "It should deserialize using the supplied converters into the custom SinkRecord shape with additional headers") public void deserialize() { try (ErrorTransform smt = new ErrorTransform()) { smt.configure( @@ -90,60 +112,30 @@ public void deserialize() { "header.converter", STRING_CONVERTER, "header.converter.converter.type", - "header")); - SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders())); - - assertThat(result.keySchema()).isNull(); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); - - // can't assert on map due to byte array equality - Map original = (Map) value.get("original"); - byte[] valueBytes = (byte[]) original.get("value"); - byte[] keyBytes = (byte[]) original.get("key"); - assertThat(valueBytes).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); - assertThat(keyBytes).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); - - assertThat(original.get("headers")).isInstanceOf(ArrayList.class); - List resultHeaders = (List) (original.get("headers")); - assertThat(resultHeaders).isNotEmpty(); - - Struct headerElement = (Struct) resultHeaders.get(0); - assertThat(headerElement.get("key")).isEqualTo("h1"); - assertThat((byte[]) headerElement.get("value")) - .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); - - assertThat(value.get("transformed")).isInstanceOf(SinkRecord.class); - SinkRecord transformed = (SinkRecord) value.get("transformed"); - assertThat(transformed.value()).isEqualTo(VALUE_STRING); - assertThat(transformed.valueSchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); - assertThat(transformed.key()).isEqualTo(KEY_STRING); - assertThat(transformed.keySchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); - assertThat(transformed.topic()).isEqualTo(TOPIC); - assertThat(transformed.kafkaPartition()).isEqualTo(PARTITION); - assertThat(transformed.kafkaOffset()).isEqualTo(OFFSET); - - ConnectHeaders expectedHeaders = new ConnectHeaders(); - expectedHeaders.add("h1", new SchemaAndValue(Schema.OPTIONAL_STRING_SCHEMA, "h1")); - assertThat(transformed.headers()).isEqualTo(expectedHeaders); - } - } - - @Test - @DisplayName("It should not have a key entry for original bytes if the key was null") - public void nullKey() { - try (ErrorTransform smt = new ErrorTransform()) { - smt.configure( - ImmutableMap.of("value.converter", STRING_CONVERTER, "key.converter", STRING_CONVERTER)); - SinkRecord result = smt.apply(createRecord(null, VALUE_STRING, null)); + "header", + "table_name", + DEAD_LETTER_TABLE_NAME)); - assertThat(result.keySchema()).isNull(); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); - Map original = (Map) value.get("original"); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders())); - assertThat(original.containsKey("value")).isTrue(); - assertThat(original.containsKey("key")).isFalse(); + assertThat(result.keySchema()).isEqualTo(SchemaBuilder.OPTIONAL_STRING_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(SchemaBuilder.OPTIONAL_STRING_SCHEMA); + assertThat(result.key()).isEqualTo(KEY_STRING); + assertThat(result.value()).isEqualTo(VALUE_STRING); + + Headers headers = result.headers(); + Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); + Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); + Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); + + assertThat(keyHeader.value()).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(valueHeader.value()).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(serializedHeader.value()).isInstanceOf(List.class); + List resultHeaders = (List) serializedHeader.value(); + assertThat(resultHeaders.size()).isEqualTo(1); + assertThat(resultHeaders.get(0).get("key")).isEqualTo("h1k"); + assertThat(resultHeaders.get(0).get("value")) + .isEqualTo("h1v".getBytes(StandardCharsets.UTF_8)); } } @@ -163,23 +155,31 @@ public void smt() { "smts", "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", "smts.transform_text", - transformString)); - SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); + transformString, + "table_name", + DEAD_LETTER_TABLE_NAME)); - assertThat(result.value()).isInstanceOf(Map.class); - Map value = (Map) result.value(); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); - assertThat(value.get("transformed")).isInstanceOf(SinkRecord.class); - SinkRecord transformed = (SinkRecord) value.get("transformed"); + assertThat(result.value()).isInstanceOf(String.class); // each transformer appends _transformed to the original value // we are configured with two transform appenders - assertThat(transformed.value()).isEqualTo(VALUE_STRING + "_transformed_transformed"); - assertThat(transformed.key()).isEqualTo(KEY_STRING); - assertThat(transformed.keySchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); - assertThat(transformed.topic()).isEqualTo(TOPIC); - assertThat(transformed.kafkaPartition()).isEqualTo(PARTITION); - assertThat(transformed.kafkaOffset()).isEqualTo(OFFSET); - assertThat(transformed.headers()).isEqualTo(new ConnectHeaders()); + assertThat(result.value()).isEqualTo(VALUE_STRING + "_transformed_transformed"); + + assertThat(result.key()).isEqualTo(KEY_STRING); + assertThat(result.keySchema()).isEqualTo(Schema.OPTIONAL_STRING_SCHEMA); + assertThat(result.topic()).isEqualTo(TOPIC); + assertThat(result.kafkaPartition()).isEqualTo(PARTITION); + + assertThat(result.kafkaOffset()).isEqualTo(OFFSET); + Headers headers = result.headers(); + Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); + Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); + Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); + + assertThat(keyHeader.value()).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(valueHeader.value()).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(serializedHeader).isNull(); } } @@ -188,7 +188,13 @@ public void smt() { public void tombstone() { try (ErrorTransform smt = new ErrorTransform()) { smt.configure( - ImmutableMap.of("value.converter", STRING_CONVERTER, "key.converter", STRING_CONVERTER)); + ImmutableMap.of( + "value.converter", + STRING_CONVERTER, + "key.converter", + STRING_CONVERTER, + "table_name", + DEAD_LETTER_TABLE_NAME)); SinkRecord record = createRecord(null, null, null); assertThat(smt.apply(record)).isSameAs(record); } @@ -212,7 +218,10 @@ public void nullFilteredBySMT() { "smts.transform_text", transformString, "smts.null", - "true")); + "true", + "table_name", + DEAD_LETTER_TABLE_NAME)); + SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); assertThat(result).isNull(); @@ -232,20 +241,21 @@ public void keyFailed() { "header.converter", STRING_CONVERTER, "header.converter.converter.type", - "header")); + "header", + "table_name", + DEAD_LETTER_TABLE_NAME)); String malformedKey = "{\"malformed_json\"\"\"{}{}{}{}**"; SinkRecord result = smt.apply(createRecord(malformedKey, VALUE_STRING, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); assertThat(value.get("partition")).isEqualTo(PARTITION); assertThat(value.get("offset")).isEqualTo(OFFSET); - assertThat(value.get("location")).isEqualTo("KEY_CONVERTER"); assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); assertThat((byte[]) value.get("key_bytes")) @@ -257,9 +267,9 @@ public void keyFailed() { List resultHeaders = (List) (value.get("headers")); assertThat(resultHeaders).isNotEmpty(); Struct headerElement = (Struct) resultHeaders.get(0); - assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat(headerElement.get("key")).isEqualTo("h1k"); assertThat((byte[]) headerElement.get("value")) - .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + .isEqualTo("h1v".getBytes(StandardCharsets.UTF_8)); } } @@ -276,20 +286,21 @@ public void valueFailed() { "header.converter", STRING_CONVERTER, "header.converter.converter.type", - "header")); + "header", + "table_name", + DEAD_LETTER_TABLE_NAME)); String malformedValue = "{\"malformed_json\"\"\"{}{}{}{}**"; SinkRecord result = smt.apply(createRecord(KEY_STRING, malformedValue, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); assertThat(value.get("partition")).isEqualTo(PARTITION); assertThat(value.get("offset")).isEqualTo(OFFSET); - assertThat(value.get("location")).isEqualTo("VALUE_CONVERTER"); assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); assertThat((byte[]) value.get("key_bytes")) @@ -301,9 +312,9 @@ public void valueFailed() { List resultHeaders = (ArrayList) (value.get("headers")); assertThat(resultHeaders).isNotEmpty(); Struct headerElement = (Struct) resultHeaders.get(0); - assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat(headerElement.get("key")).isEqualTo("h1k"); assertThat((byte[]) headerElement.get("value")) - .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + .isEqualTo("h1v".getBytes(StandardCharsets.UTF_8)); } } @@ -328,19 +339,20 @@ public void headerFailed() { "header.converter.schemas.enable", "false", "header.converter.converter.type", - "header")); + "header", + "table_name", + DEAD_LETTER_TABLE_NAME)); SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, headers); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); assertThat(value.get("partition")).isEqualTo(PARTITION); assertThat(value.get("offset")).isEqualTo(OFFSET); - assertThat(value.get("location")).isEqualTo("HEADER_CONVERTER"); assertThat(((String) value.get("stack_trace")).contains("JsonConverter")).isTrue(); assertThat(((String) value.get("exception")).contains("DataException")).isTrue(); assertThat((byte[]) value.get("key_bytes")) @@ -375,19 +387,20 @@ public void smtFailed() { "smts", "io.tabular.iceberg.connect.transforms.TestStringTransform,io.tabular.iceberg.connect.transforms.TestStringTransform", "smts.throw", - "true")); + "true", + "table_name", + DEAD_LETTER_TABLE_NAME)); SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders()); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); assertThat(result.valueSchema().name()).isEqualTo("failed_message"); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); assertThat(value.get("partition")).isEqualTo(PARTITION); assertThat(value.get("offset")).isEqualTo(OFFSET); - assertThat(value.get("location")).isEqualTo("SMT_FAILURE"); assertThat(((String) value.get("stack_trace")).contains("smt failure")).isTrue(); assertThat(((String) value.get("exception")).contains("smt failure")).isTrue(); assertThat((byte[]) value.get("key_bytes")) @@ -399,9 +412,9 @@ public void smtFailed() { List resultHeaders = (ArrayList) (value.get("headers")); assertThat(resultHeaders).isNotEmpty(); Struct headerElement = (Struct) resultHeaders.get(0); - assertThat(headerElement.get("key")).isEqualTo("h1"); + assertThat(headerElement.get("key")).isEqualTo("h1k"); assertThat((byte[]) headerElement.get("value")) - .isEqualTo("h1".getBytes(StandardCharsets.UTF_8)); + .isEqualTo("h1v".getBytes(StandardCharsets.UTF_8)); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 454641b1..3933cba7 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,7 +83,13 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; - private static final String DEAD_LETTER_TABLE_PROP = "iceberg.tables.deadletter"; + private static final String WRITE_EXCEPTION_HANDLER_PROP = "iceberg.tables.deadletter.handler"; + private static final String FAILED_RECORD_FACTORY_PROP = + "iceberg.tables.deadletter.record_factory"; + private static final String FAILED_RECORD_FACTORY_PREFIX = + "iceberg.tables.deadletter.record_factory"; + private static final String FAILED_RECORD_FACTORY_DEFAULT = + "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -93,14 +99,11 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String COMMIT_THREADS_PROP = "iceberg.control.commit.threads"; private static final String CONNECT_GROUP_ID_PROP = "iceberg.connect.group-id"; private static final String HADDOP_CONF_DIR_PROP = "iceberg.hadoop-conf-dir"; - private static final String NAME_PROP = "name"; private static final String BOOTSTRAP_SERVERS_PROP = "bootstrap.servers"; - private static final String DEFAULT_CATALOG_NAME = "iceberg"; private static final String DEFAULT_CONTROL_TOPIC = "control-iceberg"; public static final String DEFAULT_CONTROL_GROUP_PREFIX = "cg-control-"; - public static final int SCHEMA_UPDATE_RETRIES = 2; // 3 total attempts public static final int CREATE_TABLE_RETRIES = 2; // 3 total attempts @@ -239,11 +242,17 @@ private static ConfigDef newConfigDef() { Importance.MEDIUM, "Coordinator threads to use for table commits, default is (cores * 2)"); configDef.define( - DEAD_LETTER_TABLE_PROP, + WRITE_EXCEPTION_HANDLER_PROP, Type.STRING, null, Importance.MEDIUM, - "If using ErrorTransform for Dead Letter Table, the db.name to write"); + "If writing to Dead Letter Table, write exception handler class to use"); + configDef.define( + FAILED_RECORD_FACTORY_PROP, + Type.STRING, + FAILED_RECORD_FACTORY_DEFAULT, + Importance.MEDIUM, + "If writing to Dead Letter Table, failed record factory class to use"); return configDef; } @@ -341,12 +350,15 @@ public boolean dynamicTablesEnabled() { } public boolean deadLetterTableEnabled() { - String table = getString(DEAD_LETTER_TABLE_PROP); - return table != null; + return getWriteExceptionHandler() != null; + } + + public String getWriteExceptionHandler() { + return getString(WRITE_EXCEPTION_HANDLER_PROP); } - public String deadLetterTableName() { - return getString(DEAD_LETTER_TABLE_PROP); + public String getFailedRecordHandler() { + return getString(FAILED_RECORD_FACTORY_PROP); } public String tablesRouteField() { @@ -365,6 +377,10 @@ public String tablesDefaultPartitionBy() { return getString(TABLES_DEFAULT_PARTITION_BY); } + public Map failedRecordHandlerProperties() { + return PropertyUtil.propertiesWithPrefix(originalProps, FAILED_RECORD_FACTORY_PREFIX + "."); + } + public TableSinkConfig tableConfig(String tableName) { return tableConfigMap.computeIfAbsent( tableName, diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 9833b7dc..2c40cb6f 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -24,10 +24,9 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.data.IcebergWriterFactory; import io.tabular.iceberg.connect.data.Offset; -import io.tabular.iceberg.connect.data.RecordWriter; -import io.tabular.iceberg.connect.data.Utilities; +import io.tabular.iceberg.connect.data.RecordRouter; +import io.tabular.iceberg.connect.data.WriterManager; import io.tabular.iceberg.connect.data.WriterResult; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; import io.tabular.iceberg.connect.events.CommitResponsePayload; @@ -35,7 +34,6 @@ import io.tabular.iceberg.connect.events.EventType; import io.tabular.iceberg.connect.events.TableName; import io.tabular.iceberg.connect.events.TopicPartitionOffset; -import java.io.UncheckedIOException; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -43,262 +41,22 @@ import java.util.Map.Entry; import java.util.UUID; import java.util.concurrent.ExecutionException; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; public class Worker extends Channel { - private final IcebergSinkConfig config; private final SinkTaskContext context; private final String controlGroupId; - private final Map writers; + private final WriterManager writers; private final Map sourceOffsets; - private final RecordRouter recordRouter; - - public interface WriterForTable { - void write(String tableName, SinkRecord record, boolean ignoreMissingTable); - - void writeFailed(SinkRecord sample, String location, Throwable error, String targetTableName); - } - - public static class BaseWriterForTable implements WriterForTable { - - private final IcebergWriterFactory writerFactory; - private final Map writers; - - BaseWriterForTable(IcebergWriterFactory writerFactory, Map writers) { - this.writerFactory = writerFactory; - this.writers = writers; - } - - @Override - public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { - writers - .computeIfAbsent( - tableName, - notUsed -> writerFactory.createWriter(tableName, record, ignoreMissingTable)) - .write(record); - } - - @Override - public void writeFailed( - SinkRecord sample, String location, Throwable error, String targetTableName) { - throw new IllegalArgumentException("BaseWriterForTable cannot write failed records", error); - } - } - - public static class DeadLetterWriterForTable implements WriterForTable { - private static final String PAYLOAD_KEY = "transformed"; - private static final String ICEBERG_TRANSFORMATION_LOCATION = "ICEBERG_TRANSFORM"; - private final IcebergWriterFactory writerFactory; - private final Map writers; - private final String deadLetterTableName; - - private final String rowIdentifier; - - DeadLetterWriterForTable( - IcebergWriterFactory writerFactory, - Map writers, - IcebergSinkConfig config) { - this.writerFactory = writerFactory; - this.writers = writers; - Preconditions.checkNotNull( - config.deadLetterTableName(), "Dead letter table name cannot be null"); - Preconditions.checkArgument( - !config.deadLetterTableName().isEmpty(), "Dead letter table name cannot be empty"); - this.deadLetterTableName = config.deadLetterTableName().toLowerCase(); - this.rowIdentifier = config.connectorName().toLowerCase().replace('-', '_'); - } - - @SuppressWarnings("unchecked") - @Override - public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { - if (record.value() != null) { - - RecordWriter writer = null; - SinkRecord recordToWrite = null; - - if (record.value() instanceof Map) { - Map payload = (Map) record.value(); - SinkRecord transformed = (SinkRecord) payload.get(PAYLOAD_KEY); - try { - writer = - writers.computeIfAbsent( - tableName, - notUsed -> - writerFactory.createWriter(tableName, transformed, ignoreMissingTable)); - recordToWrite = transformed; - } catch (DeadLetterUtils.DeadLetterException error) { - writeFailed(record, error.getLocation(), error.getError(), tableName); - } - } else if (record.value() instanceof Struct) { - if (isFailed(record)) { - Struct transformedStruct = (Struct) record.value(); - transformedStruct.put("target_table", tableName); - transformedStruct.put("identifier", rowIdentifier); - writer = - writers.computeIfAbsent( - deadLetterTableName, - notUsed -> - writerFactory.createWriter( - deadLetterTableName, record, ignoreMissingTable)); - recordToWrite = record; - } - } else { - throw new IllegalArgumentException("Record not in format expected for dead letter table"); - } - - if (recordToWrite != null) { - try { - writer.write(recordToWrite); - } catch (UncheckedIOException error) { - throw error; - } catch (Exception error) { - SinkRecord newRecord = - DeadLetterUtils.mapToFailedRecord( - tableName, record, ICEBERG_TRANSFORMATION_LOCATION, error, rowIdentifier); - writers - .computeIfAbsent( - deadLetterTableName, - notUsed -> - writerFactory.createWriter( - deadLetterTableName, newRecord, ignoreMissingTable)) - .write(newRecord); - } - } - } - } - - @Override - public void writeFailed( - SinkRecord sample, String location, Throwable error, String targetTableName) { - SinkRecord newRecord = - DeadLetterUtils.mapToFailedRecord( - targetTableName, sample, location, error, rowIdentifier); - writers - .computeIfAbsent( - deadLetterTableName, - notUsed -> writerFactory.createWriter(deadLetterTableName, newRecord, false)) - .write(newRecord); - } - - private boolean isFailed(SinkRecord record) { - Map parameters = record.valueSchema().parameters(); - if (parameters != null) { - String isFailed = parameters.get("isFailed"); - if (isFailed != null) { - return isFailed.equals("true"); - } - } - return false; - } - } - - private abstract static class RecordRouter { - - void write(SinkRecord record) {} - } - - private static class ConfigRecordRouter extends RecordRouter { - private final WriterForTable writerForTable; - private final List tables; - - ConfigRecordRouter(WriterForTable writerForTable, List tables) { - this.writerForTable = writerForTable; - this.tables = tables; - } - - @Override - public void write(SinkRecord record) { - // route to all tables - tables.forEach( - tableName -> { - writerForTable.write(tableName, record, false); - }); - } - } - - private static class ErrorHandlingRecordRouter extends RecordRouter { - private final RecordRouter underlying; - private final WriterForTable writerForTable; - ErrorHandlingRecordRouter(RecordRouter underlying, WriterForTable writerForTable) { - this.underlying = underlying; - this.writerForTable = writerForTable; - } - - @Override - public void write(SinkRecord record) { - try { - underlying.write(record); - } catch (DeadLetterUtils.DeadLetterException e) { - writerForTable.writeFailed(record, e.getLocation(), e.getError(), null); - } - } - } - - private class StaticRecordRouter extends RecordRouter { - private final WriterForTable writerForTable; - private final String routeField; - - private final RouteExtractor extractor; - - StaticRecordRouter(WriterForTable writerForTable, String routeField, RouteExtractor extractor) { - this.writerForTable = writerForTable; - this.routeField = routeField; - this.extractor = extractor; - } - - @Override - public void write(SinkRecord record) { - String routeValue = extractor.extract(record.value(), routeField); - if (routeValue != null) { - config - .tables() - .forEach( - tableName -> - config - .tableConfig(tableName) - .routeRegex() - .ifPresent( - regex -> { - if (regex.matcher(routeValue).matches()) { - writerForTable.write(tableName, record, false); - } - })); - } - } - } - - private static class DynamicRecordRouter extends RecordRouter { - private final WriterForTable writerForTable; - private final String routeField; - private final RouteExtractor extractor; - - DynamicRecordRouter( - WriterForTable writerForTable, String routeField, RouteExtractor extractor) { - this.writerForTable = writerForTable; - this.routeField = routeField; - this.extractor = extractor; - } - - @Override - public void write(SinkRecord record) { - String routeValue = extractor.extract(record.value(), routeField); - if (routeValue != null) { - String tableName = routeValue.toLowerCase(); - writerForTable.write(tableName, record, true); - } - } - } + private final RecordRouter recordRouter; public Worker( IcebergSinkConfig config, @@ -313,76 +71,12 @@ public Worker( clientFactory); this.config = config; + this.writers = new WriterManager(writerFactory); this.context = context; this.controlGroupId = config.controlGroupId(); - this.writers = Maps.newHashMap(); - - RouteExtractor routeExtractor; - RecordRouter baseRecordRouter; - - WriterForTable writerForTable; - if (config.deadLetterTableEnabled()) { - writerForTable = new DeadLetterWriterForTable(writerFactory, this.writers, config); - routeExtractor = new ErrorHandlingRouteExtractor(new DefaultRouteExtractor()); - } else { - writerForTable = new BaseWriterForTable(writerFactory, this.writers); - routeExtractor = new DefaultRouteExtractor(); - } - - if (config.dynamicTablesEnabled()) { - Preconditions.checkNotNull( - config.tablesRouteField(), "Route field cannot be null with dynamic routing"); - baseRecordRouter = - new DynamicRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); - } else { - if (config.tablesRouteField() == null) { - // validate all table identifiers are valid, otherwise exception is thrown - // as this is an invalid config setting, not an error during processing - config.tables().forEach(TableIdentifier::of); - baseRecordRouter = new ConfigRecordRouter(writerForTable, config.tables()); - } else { - baseRecordRouter = - new StaticRecordRouter(writerForTable, config.tablesRouteField(), routeExtractor); - } - } - if (config.deadLetterTableEnabled()) { - recordRouter = new ErrorHandlingRecordRouter(baseRecordRouter, writerForTable); - } else { - recordRouter = baseRecordRouter; - } - this.sourceOffsets = Maps.newHashMap(); - } - - private interface RouteExtractor { - String extract(Object recordValue, String fieldName); - } - - private static class DefaultRouteExtractor implements RouteExtractor { - - public String extract(Object recordValue, String routeField) { - if (recordValue == null) { - return null; - } - Object routeValue = Utilities.extractFromRecordValue(recordValue, routeField); - return routeValue == null ? null : routeValue.toString(); - } - } - - private static class ErrorHandlingRouteExtractor implements RouteExtractor { - private final RouteExtractor underlying; - - ErrorHandlingRouteExtractor(RouteExtractor underlying) { - this.underlying = underlying; - } - - public String extract(Object recordValue, String routeField) { - try { - return underlying.extract(recordValue, routeField); - } catch (Exception error) { - throw new DeadLetterUtils.DeadLetterException("ROUTE_FIELD", error); - } - } + this.recordRouter = + RecordRouter.from(writers, config, this.getClass().getClassLoader(), context); } public void syncCommitOffsets() { @@ -414,8 +108,7 @@ protected boolean receive(Envelope envelope) { return false; } - List writeResults = - writers.values().stream().flatMap(writer -> writer.complete().stream()).collect(toList()); + List writeResults = writers.writeResults(); Map offsets = Maps.newHashMap(sourceOffsets); writers.clear(); @@ -470,7 +163,7 @@ protected boolean receive(Envelope envelope) { @Override public void stop() { super.stop(); - writers.values().forEach(RecordWriter::close); + writers.stop(); } public void save(Collection sinkRecords) { @@ -478,8 +171,7 @@ public void save(Collection sinkRecords) { } private void save(SinkRecord record) { - // the consumer stores the offsets that corresponds to the next record to consume, - // so increment the record offset by one + sourceOffsets.put( new TopicPartition(record.topic(), record.kafkaPartition()), new Offset(record.kafkaOffset() + 1, record.timestamp())); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java new file mode 100644 index 00000000..51cf9527 --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BaseCatalogApi implements CatalogApi { + private static final Logger LOG = LoggerFactory.getLogger(BaseCatalogApi.class); + private final Catalog catalog; + private final BiFunction schemaFactory; + private final IcebergSinkConfig config; + + BaseCatalogApi(Catalog catalog, IcebergSinkConfig config) { + this.config = config; + this.catalog = catalog; + this.schemaFactory = + (tableIdentifier, sample) -> { + Types.StructType structType; + if (sample.valueSchema() == null) { + structType = + SchemaUtils.inferIcebergType(sample.value(), config) + .orElseThrow( + () -> new DataException("Unable to create table from empty object")) + .asStructType(); + } else { + structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); + } + + return new org.apache.iceberg.Schema(structType.fields()); + }; + } + + @VisibleForTesting + BaseCatalogApi( + Catalog catalog, + IcebergSinkConfig config, + BiFunction schemaFactory) { + this.config = config; + this.catalog = catalog; + this.schemaFactory = schemaFactory; + } + + @Override + public TableIdentifier tableId(String name) { + TableIdentifier tableId; + try { + Preconditions.checkArgument(!name.isEmpty()); + tableId = TableIdentifier.parse(name); + } catch (Exception error) { + throw new WriteException.TableIdentifierException(name, error); + } + return tableId; + } + + @Override + public final Table loadTable(TableIdentifier identifier) { + try { + return catalog.loadTable(identifier); + } catch (NoSuchTableException error) { + throw error; + } catch (Exception error) { + throw new WriteException.LoadTableException(identifier, error); + } + } + + @Override + public final PartitionSpec partitionSpec(String tableName, Schema schema) { + List partitionBy = config.tableConfig(tableName).partitionBy(); + + PartitionSpec spec; + try { + spec = SchemaUtils.createPartitionSpec(schema, partitionBy); + } catch (Exception e) { + LOG.error( + "Unable to create partition spec {}, table {} will be unpartitioned", + partitionBy, + tableName, + e); + spec = PartitionSpec.unpartitioned(); + } + return spec; + } + + @Override + public Table createTable( + TableIdentifier identifier, + Schema schema, + PartitionSpec spec, + Map properties) { + + try { + return catalog.createTable(identifier, schema, spec, properties); + } catch (Exception error) { + throw new WriteException.CreateTableException(identifier, error); + } + } + + @Override + public Schema schema(TableIdentifier identifier, SinkRecord sample) { + try { + return schemaFactory.apply(identifier, sample); + } catch (Exception error) { + throw new WriteException.CreateSchemaException(identifier, error); + } + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java index 1658ac4a..8eab4b5a 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/CatalogApi.java @@ -18,151 +18,26 @@ */ package io.tabular.iceberg.connect.data; -import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; -import java.util.List; import java.util.Map; -import java.util.function.BiFunction; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.types.Types; -import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public abstract class CatalogApi { - private static final Logger LOG = LoggerFactory.getLogger(CatalogApi.class); +public interface CatalogApi { - private final Catalog catalog; - private final IcebergSinkConfig config; + TableIdentifier tableId(String name); - CatalogApi(Catalog catalog, IcebergSinkConfig config) { - this.catalog = catalog; - this.config = config; - } + Table loadTable(TableIdentifier identifier); - TableIdentifier tableId(String name) { - return TableIdentifier.parse(name); - } + PartitionSpec partitionSpec(String tableName, Schema schema); - public final Table loadTable(TableIdentifier identifier) { - return catalog.loadTable(identifier); - } - - public final PartitionSpec partitionSpec(String tableName, Schema schema) { - List partitionBy = config.tableConfig(tableName).partitionBy(); - - PartitionSpec spec; - try { - spec = SchemaUtils.createPartitionSpec(schema, partitionBy); - } catch (Exception e) { - LOG.error( - "Unable to create partition spec {}, table {} will be unpartitioned", - partitionBy, - tableName, - e); - spec = PartitionSpec.unpartitioned(); - } - return spec; - } - - public Table createTable( + Table createTable( TableIdentifier identifier, Schema schema, PartitionSpec spec, - Map properties) { - return catalog.createTable(identifier, schema, spec, properties); - } - - public Schema schema(TableIdentifier identifier, SinkRecord sample) { - Types.StructType structType; - if (sample.valueSchema() == null) { - structType = - SchemaUtils.inferIcebergType(sample.value(), config) - .orElseThrow(() -> new DataException("Unable to create table from empty object")) - .asStructType(); - } else { - structType = SchemaUtils.toIcebergType(sample.valueSchema(), config).asStructType(); - } - - return new org.apache.iceberg.Schema(structType.fields()); - } - - public static class ErrorHandlingCatalogApi extends CatalogApi { - - private final TableIdentifier deadLetterTableId; - private final Catalog catalog; - - private final BiFunction schemaFactory; - - ErrorHandlingCatalogApi(Catalog catalog, IcebergSinkConfig config) { - super(catalog, config); - this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); - this.catalog = catalog; - this.schemaFactory = super::schema; - } - - @VisibleForTesting - ErrorHandlingCatalogApi( - Catalog catalog, - IcebergSinkConfig config, - BiFunction schemaFactory) { - super(catalog, config); - this.deadLetterTableId = TableIdentifier.parse(config.deadLetterTableName()); - this.catalog = catalog; - this.schemaFactory = schemaFactory; - } - - @Override - TableIdentifier tableId(String name) { - TableIdentifier tableId; - try { - tableId = super.tableId(name); - } catch (Exception error) { - throw new DeadLetterUtils.DeadLetterException("TABLE_IDENTIFIER", error); - } - return tableId; - } - - @Override - public Table createTable( - TableIdentifier identifier, - Schema schema, - PartitionSpec spec, - Map properties) { - - Table table; - if (identifier == deadLetterTableId) { - table = catalog.createTable(identifier, schema, spec, properties); - } else { - try { - table = catalog.createTable(identifier, schema, spec, properties); - } catch (IllegalArgumentException | ValidationException error) { - throw new DeadLetterUtils.DeadLetterException("CREATE_TABLE", error); - } - } - return table; - } + Map properties); - @Override - public Schema schema(TableIdentifier identifier, SinkRecord sample) { - Schema schema; - if (identifier == deadLetterTableId) { - schema = this.schemaFactory.apply(identifier, sample); - } else { - try { - schema = this.schemaFactory.apply(identifier, sample); - } catch (IllegalArgumentException | ValidationException error) { - throw new DeadLetterUtils.DeadLetterException("CREATE_SCHEMA", error); - } - } - return schema; - } - } + Schema schema(TableIdentifier identifier, SinkRecord sample); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java new file mode 100644 index 00000000..4aca6140 --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +public class DefaultWriteExceptionHandler implements WriteExceptionHandler { + private FailedRecordFactory factory; + + @Override + public void initialize( + SinkTaskContext context, IcebergSinkConfig config, FailedRecordFactory recordFactory) { + this.factory = recordFactory; + } + + @Override + public Result handle(SinkRecord record, Exception exception) { + if (exception instanceof WriteException) { + return handleWriteException(record, (WriteException) exception); + } + Throwable cause = exception.getCause(); + if (cause instanceof WriteException) { + return handleWriteException(record, (WriteException) cause); + } + throw new RuntimeException(exception); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private Result handleWriteException(SinkRecord record, WriteException exception) { + if (exception instanceof WriteException.CreateTableException) { + Throwable cause = exception.getCause(); + if (cause instanceof IllegalArgumentException || cause instanceof ValidationException) { + return failedRecord(record, exception); + } + } else if (exception instanceof WriteException.CreateSchemaException) { + return failedRecord(record, exception); + } else if (exception instanceof WriteException.LoadTableException) { + Throwable cause = exception.getCause(); + if (cause instanceof IllegalArgumentException || cause instanceof ValidationException) { + return failedRecord(record, exception); + } + } else if (exception instanceof WriteException.RecordConversionException) { + return failedRecord(record, exception); + + } else if (exception instanceof WriteException.RouteException) { + return failedRecord(record, exception); + + } else if (exception instanceof WriteException.RouteRegexException) { + return failedRecord(record, exception); + + } else if (exception instanceof WriteException.SchemaEvolutionException) { + Throwable cause = exception.getCause(); + if (cause instanceof IllegalArgumentException + || cause instanceof ValidationException + || cause instanceof UnsupportedOperationException) { + return failedRecord(record, exception); + } + } else if (exception instanceof WriteException.TableIdentifierException) { + return failedRecord(record, exception); + } + throw exception; + } + + private Result failedRecord(SinkRecord record, WriteException exception) { + String targetTableId = exception.tableId(); + if (targetTableId != null && targetTableId.equals(factory.tableName(record))) { + throw new IllegalArgumentException( + String.format( + "Must throw for exceptions involving target Dead Letter Table: %s", targetTableId), + exception); + } + return new Result(factory.recordFromConnector(record, exception, null), targetTableId); + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java index 9b81034c..e9cff67d 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java @@ -92,6 +92,7 @@ private Record convertToRow(SinkRecord record) { // initialize a new writer with the new schema initNewWriter(); // convert the row again, this time using the new table schema + // fail here again row = recordConverter.convert(record.value(), null); } @@ -99,8 +100,12 @@ private Record convertToRow(SinkRecord record) { } private Operation extractCdcOperation(Object recordValue, String cdcField) { - Object opValue = Utilities.extractFromRecordValue(recordValue, cdcField); - + Object opValue; + try { + opValue = Utilities.extractFromRecordValue(recordValue, cdcField); + } catch (Exception e) { + throw new WriteException.CdcException(e); + } if (opValue == null) { return Operation.INSERT; } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index 29173336..c6843279 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -20,7 +20,6 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Predicate; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -35,8 +34,6 @@ public class IcebergWriterFactory { private final IcebergSinkConfig config; - private final Predicate shouldAutoCreate; - private final CatalogApi catalogApi; public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { @@ -46,31 +43,21 @@ public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { public IcebergWriterFactory(IcebergSinkConfig config, CatalogApi api) { this.config = config; this.catalogApi = api; - - if (config.autoCreateEnabled()) { - shouldAutoCreate = (unused) -> true; - } else if (config.deadLetterTableEnabled()) { - String deadLetterTableName = config.deadLetterTableName().toLowerCase(); - shouldAutoCreate = (tableName) -> tableName.equals(deadLetterTableName); - } else { - shouldAutoCreate = (unused) -> false; - } } public RecordWriter createWriter( String tableName, SinkRecord sample, boolean ignoreMissingTable) { - - TableIdentifier identifier = catalogApi.tableId(tableName); + TableIdentifier identifier = TableIdentifier.parse(tableName); Table table; try { table = catalogApi.loadTable(identifier); } catch (NoSuchTableException nst) { - if (shouldAutoCreate.test(tableName)) { + if (config.autoCreateEnabled()) { table = autoCreateTable(tableName, sample); } else if (ignoreMissingTable) { return new RecordWriter() {}; } else { - throw nst; + throw new WriteException.LoadTableException(identifier, nst); } } @@ -79,7 +66,6 @@ public RecordWriter createWriter( @VisibleForTesting Table autoCreateTable(String tableName, SinkRecord sample) { - TableIdentifier identifier = catalogApi.tableId(tableName); Schema schema = catalogApi.schema(identifier, sample); @@ -103,10 +89,6 @@ Table autoCreateTable(String tableName, SinkRecord sample) { } private static CatalogApi getCatalogApi(Catalog catalog, IcebergSinkConfig config) { - if (config.deadLetterTableEnabled()) { - return new CatalogApi.ErrorHandlingCatalogApi(catalog, config); - } else { - return new CatalogApi(catalog, config) {}; - } + return new BaseCatalogApi(catalog, config) {}; } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java index b62bd1a7..5809456f 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java @@ -91,10 +91,15 @@ public Record convert(Object data) { } public Record convert(Object data, SchemaUpdate.Consumer schemaUpdateConsumer) { - if (data instanceof Struct || data instanceof Map) { - return convertStructValue(data, tableSchema.asStruct(), -1, schemaUpdateConsumer); + try { + if (data instanceof Struct || data instanceof Map) { + return convertStructValue(data, tableSchema.asStruct(), -1, schemaUpdateConsumer); + } + throw new WriteException.RecordConversionException( + new UnsupportedOperationException("Cannot convert type: " + data.getClass().getName())); + } catch (Exception error) { + throw new WriteException.RecordConversionException(error); } - throw new UnsupportedOperationException("Cannot convert type: " + data.getClass().getName()); } private NameMapping createNameMapping(Table table) { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java new file mode 100644 index 00000000..f5a5c59e --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import java.util.List; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +public abstract class RecordRouter { + + public void write(SinkRecord record) {} + + protected final String extractRouteValue(Object recordValue, String routeField) { + Object routeValue; + if (recordValue == null) { + return null; + } + try { + routeValue = Utilities.extractFromRecordValue(recordValue, routeField); + } catch (Exception error) { + throw new WriteException.RouteException(error); + } + return routeValue == null ? null : routeValue.toString(); + } + + public static RecordRouter from( + WriterManager writers, + IcebergSinkConfig config, + ClassLoader loader, + SinkTaskContext context) { + RecordRouter baseRecordRouter; + + if (config.dynamicTablesEnabled()) { + Preconditions.checkNotNull( + config.tablesRouteField(), "Route field cannot be null with dynamic routing"); + baseRecordRouter = new DynamicRecordRouter(writers, config.tablesRouteField()); + } else { + if (config.tablesRouteField() == null) { + // validate all table identifiers are valid, otherwise exception is thrown + // as this is an invalid config setting, not an error during processing + config.tables().forEach(TableIdentifier::of); + baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); + } else { + baseRecordRouter = new StaticRecordRouter(writers, config); + } + } + + if (config.deadLetterTableEnabled()) { + String failedRecordFactoryClass = config.getFailedRecordHandler(); + String handlerClass = config.getWriteExceptionHandler(); + FailedRecordFactory factory = + (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, loader); + factory.configure(config.failedRecordHandlerProperties()); + WriteExceptionHandler handler = + (WriteExceptionHandler) DeadLetterUtils.loadClass(handlerClass, loader); + handler.initialize(context, config, factory); + baseRecordRouter = + new RecordRouter.ErrorHandlingRecordRouter(baseRecordRouter, handler, writers, factory); + } + + return baseRecordRouter; + } + + private static class ConfigRecordRouter extends RecordRouter { + private final List tables; + private final WriterManager writers; + + ConfigRecordRouter(WriterManager writers, List tables) { + this.tables = tables; + this.writers = writers; + } + + @Override + public void write(SinkRecord record) { + // route to all tables + tables.forEach( + tableName -> { + writers.write(tableName, record, false); + }); + } + } + + private static class StaticRecordRouter extends RecordRouter { + private final String routeField; + private final WriterManager writers; + private final IcebergSinkConfig config; + + StaticRecordRouter(WriterManager writers, IcebergSinkConfig config) { + this.routeField = config.tablesRouteField(); + this.writers = writers; + this.config = config; + } + + @Override + public void write(SinkRecord record) { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + config + .tables() + .forEach( + tableName -> + config + .tableConfig(tableName) + .routeRegex() + .ifPresent( + regex -> { + boolean matches; + try { + matches = regex.matcher(routeValue).matches(); + } catch (Exception error) { + throw new WriteException.RouteRegexException(error); + } + if (matches) { + writers.write(tableName, record, false); + } + })); + } + } + } + + private static class DynamicRecordRouter extends RecordRouter { + private final String routeField; + private final WriterManager writers; + + DynamicRecordRouter(WriterManager writers, String routeField) { + this.routeField = routeField; + this.writers = writers; + } + + @Override + public void write(SinkRecord record) { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + String tableName = routeValue.toLowerCase(); + writers.write(tableName, record, true); + } + } + } + + private static class ErrorHandlingRecordRouter extends RecordRouter { + private final WriteExceptionHandler handler; + private final WriterManager writers; + private final RecordRouter router; + private final FailedRecordFactory failedRecordFactory; + + ErrorHandlingRecordRouter( + RecordRouter baseRouter, + WriteExceptionHandler handler, + WriterManager writers, + FailedRecordFactory factory) { + this.router = baseRouter; + this.handler = handler; + this.writers = writers; + this.failedRecordFactory = factory; + } + + @Override + public void write(SinkRecord record) { + if (failedRecordFactory.isFailedTransformRecord(record)) { + writers.write(failedRecordFactory.tableName(record), record, false); + } else { + try { + router.write(record); + } catch (Exception error) { + WriteExceptionHandler.Result result = handler.handle(record, error); + if (result != null) { + writers.write(result.tableName(), result.sinkRecord(), false); + } + } + } + } + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java index 43c7944b..0e97de1f 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java @@ -88,9 +88,13 @@ public static void applySchemaUpdates(Table table, SchemaUpdate.Consumer updates return; } - Tasks.range(1) - .retry(IcebergSinkConfig.SCHEMA_UPDATE_RETRIES) - .run(notUsed -> commitSchemaUpdates(table, updates)); + try { + Tasks.range(1) + .retry(IcebergSinkConfig.SCHEMA_UPDATE_RETRIES) + .run(notUsed -> commitSchemaUpdates(table, updates)); + } catch (Exception error) { + throw new WriteException.SchemaEvolutionException(table.name(), error); + } } private static void commitSchemaUpdates(Table table, SchemaUpdate.Consumer updates) { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java new file mode 100644 index 00000000..1cc41add --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import org.apache.iceberg.catalog.TableIdentifier; + +public class WriteException extends RuntimeException { + + private final String tableIdentifier; + + WriteException(Throwable cause) { + super(cause); + tableIdentifier = null; + } + + WriteException(TableIdentifier tableId, Throwable cause) { + super(cause); + this.tableIdentifier = tableId.toString(); + } + + WriteException(String tableId, Throwable cause) { + super(cause); + this.tableIdentifier = tableId; + } + + public String tableId() { + return tableIdentifier; + } + + public static class CdcException extends WriteException { + public CdcException(Throwable cause) { + super(cause); + } + } + + public static class CreateTableException extends WriteException { + + public CreateTableException(TableIdentifier identifier, Throwable cause) { + super(identifier, cause); + } + } + + public static class CreateSchemaException extends WriteException { + public CreateSchemaException(TableIdentifier identifier, Throwable cause) { + super(identifier, cause); + } + } + + public static class LoadTableException extends WriteException { + + public LoadTableException(TableIdentifier identifier, Throwable cause) { + super(identifier, cause); + } + } + + public static class RecordConversionException extends WriteException { + + RecordConversionException(Throwable cause) { + super(cause); + } + } + + public static class RouteException extends WriteException { + RouteException(Throwable cause) { + super(cause); + } + } + + public static class RouteRegexException extends WriteException { + RouteRegexException(Throwable cause) { + super(cause); + } + } + + public static class SchemaEvolutionException extends WriteException { + + SchemaEvolutionException(String name, Throwable cause) { + super(name, cause); + } + } + + public static class TableIdentifierException extends WriteException { + TableIdentifierException(String name, Throwable cause) { + super(name, cause); + } + } +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java new file mode 100644 index 00000000..d2dd4cdc --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +public interface WriteExceptionHandler { + void initialize(SinkTaskContext context, IcebergSinkConfig config, FailedRecordFactory factory); + + class Result { + private final SinkRecord sinkRecord; + private final String tableName; + + public Result(SinkRecord sinkRecord, String tableName) { + this.sinkRecord = sinkRecord; + this.tableName = tableName; + } + + public SinkRecord sinkRecord() { + return sinkRecord; + } + + public String tableName() { + return tableName; + } + } + + /** + * This method will be invoked whenever the connector runs into an exception while trying to write + * SinkRecords to a table. Implementations of this method have 3 general options: + * + *
    + *
  1. Return a SinkRecord and the name of the table to write to (wrapped inside a {@link + * Result}) + *
  2. Return null to drop the SinkRecord + *
+ * + * @param record The SinkRecord that couldn't be written + */ + Result handle(SinkRecord record, Exception exception); +} diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriterManager.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriterManager.java new file mode 100644 index 00000000..0075641a --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriterManager.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import static java.util.stream.Collectors.toList; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.connect.sink.SinkRecord; + +public class WriterManager { + private final IcebergWriterFactory writerFactory; + + private final Map writers; + + public WriterManager(IcebergWriterFactory writerFactory) { + this.writerFactory = writerFactory; + this.writers = Maps.newHashMap(); + } + + public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { + writers + .computeIfAbsent( + tableName, notUsed -> writerFactory.createWriter(tableName, record, ignoreMissingTable)) + .write(record); + } + + public List writeResults() { + return writers.values().stream() + .flatMap(writer -> writer.complete().stream()) + .collect(toList()); + } + + public void clear() { + this.writers.clear(); + } + + public void stop() { + writers.values().forEach(RecordWriter::close); + } +} diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 47e23323..12edbc38 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -19,9 +19,6 @@ package io.tabular.iceberg.connect.channel; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -29,15 +26,12 @@ import io.tabular.iceberg.connect.data.IcebergWriterFactory; import io.tabular.iceberg.connect.data.RecordWriter; import io.tabular.iceberg.connect.data.WriterResult; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; import io.tabular.iceberg.connect.events.CommitResponsePayload; import io.tabular.iceberg.connect.events.Event; import io.tabular.iceberg.connect.events.EventTestUtil; import io.tabular.iceberg.connect.events.EventType; -import java.io.IOException; -import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -47,18 +41,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; import org.mockito.ArgumentMatchers; @@ -92,33 +79,6 @@ public void write(SinkRecord record) { private static final byte[] ORIGINAL_BYTES = "{\"field\":\"success\"}".getBytes(StandardCharsets.UTF_8); - private SinkRecord errorTransformSuccessRecord() { - Schema schema = - SchemaBuilder.struct() - .field("field", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) - .build(); - Struct struct = new Struct(schema); - struct.put("field", "success"); - Map map = Maps.newHashMap(); - SinkRecord success = - new SinkRecord( - "topic", 0, null, null, schema, struct, 100L, 1000L, TimestampType.CREATE_TIME); - map.put(DeadLetterUtils.PAYLOAD_KEY, success); - Map failed = Maps.newHashMap(); - failed.put(DeadLetterUtils.VALUE_BYTES, ORIGINAL_BYTES); - map.put(DeadLetterUtils.ORIGINAL_BYTES_KEY, failed); - return new SinkRecord( - "topic", 0, null, null, null, map, 100L, 1000L, TimestampType.CREATE_TIME); - } - - private SinkRecord errorTransformFailedRecord() { - SinkRecord originalRecord = - new SinkRecord( - "topic", 0, null, null, null, ORIGINAL_BYTES, 100L, 1000L, TimestampType.CREATE_TIME); - return DeadLetterUtils.failedRecord( - originalRecord, new IllegalArgumentException("test"), "location", "test_identifier"); - } - @Test public void testStaticRoute() { when(config.tables()).thenReturn(ImmutableList.of(TABLE_NAME)); @@ -186,248 +146,4 @@ private void workerTest(Map value) { // offset should be one more than the record offset assertThat(readyPayload.assignments().get(0).offset()).isEqualTo(1L); } - - @Test - @DisplayName("BaseWriterForTable should create writers and pass records to them") - public void baseWriterForTable() { - when(config.deadLetterTableEnabled()).thenReturn(false); - - RecordingRecordWriter writer = new RecordingRecordWriter(false); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.BaseWriterForTable writerForTable = - new Worker.BaseWriterForTable(writerFactory, writerMap); - - SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, "test", 0); - writerForTable.write(TABLE_NAME, nullRecord, true); - - assertThat(writer.written.size()).isEqualTo(1); - assertThat(writer.written.get(0).value()).isEqualTo("test"); - assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(TABLE_NAME)); - } - - @Test - @DisplayName("DeadLetterWriterForTable should ignore null records") - public void deadLetterWriterForTableNullRecords() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - RecordingRecordWriter writer = new RecordingRecordWriter(false); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - SinkRecord nullRecord = new SinkRecord(SRC_TOPIC_NAME, 0, null, null, null, null, 0); - writerForTable.write(TABLE_NAME, nullRecord, true); - - assertThat(writer.written).isEmpty(); - } - - @Test - @DisplayName("DeadLetterWriterForTable should write successful ErrorTransformed records") - public void deadLetterWriterForTableSuccessErrorTransform() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - RecordingRecordWriter writer = new RecordingRecordWriter(false); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); - Schema expectedSchema = - SchemaBuilder.struct() - .field("field", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) - .build(); - assertThat(writer.written.size()).isEqualTo(1); - SinkRecord result = writer.written.get(0); - assertThat(result.valueSchema()).isEqualTo(expectedSchema); - assertThat(result.value()).isInstanceOf(Struct.class); - Struct resultStruct = (Struct) result.value(); - assertThat(resultStruct.get("field")).isEqualTo("success"); - } - - @Test - @DisplayName("DeadLetterWriterForTable should write failed ErrorTransformed records") - public void deadLetterWriterForTableFailureErrorTransform() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - RecordingRecordWriter writer = new RecordingRecordWriter(false); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - writerForTable.write(TABLE_NAME, errorTransformFailedRecord(), true); - assertThat(writer.written.size()).isEqualTo(1); - SinkRecord result = writer.written.get(0); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); - assertThat(result.value()).isInstanceOf(Struct.class); - Struct resultValue = (Struct) result.value(); - assertThat(resultValue.get("exception").toString().contains("test")).isTrue(); - String resultBytesAsString = - new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); - assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); - assertThat(resultValue.get("target_table")).isEqualTo(TABLE_NAME); - assertThat(resultValue.get("identifier")).isEqualTo("connector_name"); - } - - @Test - @DisplayName( - "DeadLetterWriterForTable should convert Iceberg Writer failures to failed SinkRecord") - public void deadLetterWriterForTableConvertWriterFailures() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - RecordingRecordWriter writer = new RecordingRecordWriter(true); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); - assertThat(writer.written.size()).isEqualTo(1); - SinkRecord result = writer.written.get(0); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); - assertThat(result.value()).isInstanceOf(Struct.class); - Struct resultValue = (Struct) result.value(); - assertThat(resultValue.get("exception").toString().contains("test throw")).isTrue(); - String resultBytesAsString = - new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); - assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); - assertThat(resultValue.get("target_table")).isEqualTo(TABLE_NAME); - assertThat(resultValue.get("identifier")).isEqualTo("connector_name"); - // we successfully create the regular writer, but fail to write it, seeing both writers in the - // Map - assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(TABLE_NAME, DEAD_LETTER_TABLE_NAME)); - } - - @Test - @DisplayName( - "DeadLetterWriterForTable should fail when non-DeadLetterExceptions are thrown during writer creation") - public void deadLetterWriterThrowWriterCreation() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenThrow(new IllegalArgumentException("writer factory throws")); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - assertThrows( - IllegalArgumentException.class, - () -> writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true)); - } - - @Test - @DisplayName( - "DeadLetterWriterForTable should write failed SinkRecords when DeadLetterWriter exceptions are thrown during writer creation") - public void deadLetterWriterDeadLetterExceptionsAtCreation() { - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - - RecordingRecordWriter writer = new RecordingRecordWriter(false); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - eq(TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenThrow( - new DeadLetterUtils.DeadLetterException( - "test", new IllegalArgumentException("creation throw"))); - when(writerFactory.createWriter( - eq(DEAD_LETTER_TABLE_NAME), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true); - - assertThat(writer.written.size()).isEqualTo(1); - SinkRecord result = writer.written.get(0); - assertThat(result.valueSchema()).isEqualTo(DeadLetterUtils.FAILED_SCHEMA); - assertThat(result.value()).isInstanceOf(Struct.class); - Struct resultValue = (Struct) result.value(); - assertThat(resultValue.get("stack_trace").toString().contains("creation throw")).isTrue(); - String resultBytesAsString = - new String((byte[]) resultValue.get("value_bytes"), StandardCharsets.UTF_8); - assertThat(resultBytesAsString).isEqualTo(new String(ORIGINAL_BYTES, StandardCharsets.UTF_8)); - // fail to create the original message, but do successfully create the dead letter table - assertThat(writerMap.keySet()).isEqualTo(Sets.newHashSet(DEAD_LETTER_TABLE_NAME)); - } - - @Test - @DisplayName( - "DeadLetterWriterForTable should fail when UncheckedIOException is thrown on writing a record") - public void deadLetterWriterThrowWriterWriteUncheckedIO() { - // the writer may close/flush during writing due to schema changes being applied to the table - // which introduces the possibility of IOExceptions - when(config.deadLetterTableEnabled()).thenReturn(true); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE_NAME); - when(config.connectorName()).thenReturn("connector-name"); - RecordWriter writer = mock(IcebergWriter.class); - - doThrow(new UncheckedIOException(new IOException("test"))) - .when(writer) - .write(ArgumentMatchers.any()); - - IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); - when(writerFactory.createWriter( - ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.anyBoolean())) - .thenReturn(writer); - - Map writerMap = Maps.newHashMap(); - - Worker.DeadLetterWriterForTable writerForTable = - new Worker.DeadLetterWriterForTable(writerFactory, writerMap, config); - - assertThrows( - UncheckedIOException.class, - () -> writerForTable.write(TABLE_NAME, errorTransformSuccessRecord(), true)); - } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java similarity index 54% rename from kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java rename to kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java index 738a4cb9..cc9522b4 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/CatalogApiTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java @@ -25,7 +25,6 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.TableSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import java.util.function.BiFunction; import java.util.regex.Pattern; import org.apache.iceberg.catalog.Catalog; @@ -41,7 +40,7 @@ import org.junit.jupiter.api.Test; import org.mockito.ArgumentMatchers; -public class CatalogApiTest { +public class BaseCatalogApiTest { private static final String DEAD_LETTER_TABLE = "dlt.table"; private static final TableIdentifier DEAD_LETTER_TABLE_ID = @@ -61,43 +60,23 @@ private SinkRecord sinkRecord() { } @Test - @DisplayName("CatalogApi.tableId throw exceptions for invalid table names") + @DisplayName("tableId throw exceptions for invalid table names") public void tableIdThrows() { Catalog catalog = mock(Catalog.class); IcebergSinkConfig config = mock(IcebergSinkConfig.class); - CatalogApi defaultCatalog = new CatalogApi(catalog, config) {}; - assertThrows(IllegalArgumentException.class, () -> defaultCatalog.tableId("")); + CatalogApi catalogApi = new BaseCatalogApi(catalog, config) {}; + assertThrows(WriteException.TableIdentifierException.class, () -> catalogApi.tableId("")); } @Test - @DisplayName( - "ErrorHandlingCatalogApi.tableId should throw DeadLetterExceptions for invalid table names") - public void errorTableIdThrows() { - Catalog catalog = mock(Catalog.class); - IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); - CatalogApi errorHandlingCatalog = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); - assertThrows(DeadLetterUtils.DeadLetterException.class, () -> errorHandlingCatalog.tableId("")); - } - - @Test - @DisplayName( - "ErrorHandlingCatalog constructor should throw if dead letter table has invalid name") - public void errorHandlingCatalogConstructorThrows() { - Catalog catalog = mock(Catalog.class); - IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(""); - assertThrows( - IllegalArgumentException.class, - () -> new CatalogApi.ErrorHandlingCatalogApi(catalog, config)); - } + @DisplayName("loadTable should throw LoadTable exceptions wrapping underlying exceptions") + public void loadTableThrows() {} @Test - @DisplayName("ErrorHandlingCatalogAPI.schema should wrap validation/illegal argument exceptions") - public void errorHandlingCatalogSchemaShouldWrap() { + @DisplayName("schema should wrap exceptions") + public void catalogApiSchemaShouldWrap() { Catalog catalog = mock(Catalog.class); IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); BiFunction illegalArgFn = (a, b) -> { @@ -109,25 +88,22 @@ public void errorHandlingCatalogSchemaShouldWrap() { throw new ValidationException("test"); }; - CatalogApi catalogApiIllegalArg = - new CatalogApi.ErrorHandlingCatalogApi(catalog, config, illegalArgFn); - CatalogApi catalogApiValidationExp = - new CatalogApi.ErrorHandlingCatalogApi(catalog, config, validationExceptionFn); + CatalogApi catalogApiIllegalArg = new BaseCatalogApi(catalog, config, illegalArgFn); + CatalogApi catalogApiValidationExp = new BaseCatalogApi(catalog, config, validationExceptionFn); assertThrows( - DeadLetterUtils.DeadLetterException.class, + WriteException.CreateSchemaException.class, () -> catalogApiIllegalArg.schema(DEAD_LETTER_TABLE_ID, sinkRecord())); assertThrows( - DeadLetterUtils.DeadLetterException.class, + WriteException.CreateSchemaException.class, () -> catalogApiValidationExp.schema(DEAD_LETTER_TABLE_ID, sinkRecord())); } @Test - @DisplayName("CatalogAPI/Error.partitionSpec should apply the configured PartitionSpec") + @DisplayName("partitionSpec should apply the configured PartitionSpec") public void catalogApiAppliesPartitionConfig() { Catalog catalog = mock(Catalog.class); IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); TableSinkConfig tableConfig = new TableSinkConfig( Pattern.compile(".*123", Pattern.DOTALL), @@ -136,20 +112,17 @@ public void catalogApiAppliesPartitionConfig() { null); when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); - CatalogApi catalogApi = new CatalogApi(catalog, config) {}; - CatalogApi errorApi = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + CatalogApi catalogApi = new BaseCatalogApi(catalog, config) {}; org.apache.iceberg.Schema schema = catalogApi.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); assertThat(catalogApi.partitionSpec(DEAD_LETTER_TABLE, schema).isPartitioned()).isTrue(); - assertThat(errorApi.partitionSpec(DEAD_LETTER_TABLE, schema).isPartitioned()).isTrue(); } @Test - @DisplayName("CatalogAPI/Error.partitionSpec should create be unpartitioned if an error occurs") + @DisplayName(".partitionSpec should create be unpartitioned if an error occurs") public void catalogApiPartitionSpecUnpartitioned() { Catalog catalog = mock(Catalog.class); IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); // partition on a field that does not exist TableSinkConfig tableConfig = new TableSinkConfig( @@ -159,16 +132,14 @@ public void catalogApiPartitionSpecUnpartitioned() { null); when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); - CatalogApi catalogApi = new CatalogApi(catalog, config) {}; - CatalogApi errorApi = new CatalogApi.ErrorHandlingCatalogApi(catalog, config); + CatalogApi catalogApi = new BaseCatalogApi(catalog, config) {}; org.apache.iceberg.Schema schema = catalogApi.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); assertThat(catalogApi.partitionSpec(DEAD_LETTER_TABLE, schema).isUnpartitioned()).isTrue(); - assertThat(errorApi.partitionSpec(DEAD_LETTER_TABLE, schema).isUnpartitioned()).isTrue(); } @Test - @DisplayName("CatalogAPI.createTable should throw validation/illegal argument exceptions") + @DisplayName("createTable should throw CreateTable exceptions for underlying exceptions") public void catalogCreateTableShouldThrow() { Catalog catalogValidationException = mock(Catalog.class); when(catalogValidationException.createTable( @@ -187,61 +158,6 @@ public void catalogCreateTableShouldThrow() { .thenThrow(new IllegalArgumentException("test")); IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); - TableSinkConfig tableConfig = - new TableSinkConfig( - Pattern.compile(".*123", Pattern.DOTALL), - Lists.newArrayList(), - Lists.newArrayList("a"), - null); - - when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); - CatalogApi catalogApiValidation = new CatalogApi(catalogValidationException, config) {}; - CatalogApi catalogApiIllegal = new CatalogApi(catalogIllegalArgException, config) {}; - - org.apache.iceberg.Schema schema = - catalogApiValidation.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); - - assertThrows( - ValidationException.class, - () -> - catalogApiValidation.createTable( - DEAD_LETTER_TABLE_ID, - schema, - catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), - Maps.newHashMap())); - assertThrows( - IllegalArgumentException.class, - () -> - catalogApiIllegal.createTable( - DEAD_LETTER_TABLE_ID, - schema, - catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), - Maps.newHashMap())); - } - - @Test - @DisplayName( - "ErrorHandlingCatalogApi.createTable should wrap validation/illegal argument exceptions") - public void errorHandlingCatalogCreateTableShouldWrap() { - Catalog catalogValidationException = mock(Catalog.class); - when(catalogValidationException.createTable( - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any())) - .thenThrow(new ValidationException("test")); - - Catalog catalogIllegalArgException = mock(Catalog.class); - when(catalogIllegalArgException.createTable( - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any(), - ArgumentMatchers.any())) - .thenThrow(new IllegalArgumentException("test")); - - IcebergSinkConfig config = mock(IcebergSinkConfig.class); - when(config.deadLetterTableName()).thenReturn(DEAD_LETTER_TABLE); TableSinkConfig tableConfig = new TableSinkConfig( Pattern.compile(".*123", Pattern.DOTALL), @@ -250,16 +166,14 @@ public void errorHandlingCatalogCreateTableShouldWrap() { null); when(config.tableConfig(ArgumentMatchers.any())).thenReturn(tableConfig); - CatalogApi catalogApiValidation = - new CatalogApi.ErrorHandlingCatalogApi(catalogValidationException, config) {}; - CatalogApi catalogApiIllegal = - new CatalogApi.ErrorHandlingCatalogApi(catalogIllegalArgException, config) {}; + CatalogApi catalogApiValidation = new BaseCatalogApi(catalogValidationException, config) {}; + CatalogApi catalogApiIllegal = new BaseCatalogApi(catalogIllegalArgException, config) {}; org.apache.iceberg.Schema schema = catalogApiValidation.schema(DEAD_LETTER_TABLE_ID, sinkRecord()); assertThrows( - DeadLetterUtils.DeadLetterException.class, + WriteException.CreateTableException.class, () -> catalogApiValidation.createTable( DEAD_LETTER_TABLE_ID, @@ -267,7 +181,7 @@ public void errorHandlingCatalogCreateTableShouldWrap() { catalogApiValidation.partitionSpec(DEAD_LETTER_TABLE, schema), Maps.newHashMap())); assertThrows( - DeadLetterUtils.DeadLetterException.class, + WriteException.CreateTableException.class, () -> catalogApiIllegal.createTable( DEAD_LETTER_TABLE_ID, diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java index d1577813..b15e7394 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/IcebergWriterFactoryTest.java @@ -43,8 +43,6 @@ public class IcebergWriterFactoryTest { - private static final String DEAD_LETTER_TABLE = "dlt.name"; - @ParameterizedTest @ValueSource(booleans = {true, false}) @SuppressWarnings("unchecked") From 01f8cbb423804fcdd201acc3e4d18c6b5f7fa13b Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 26 Apr 2024 16:49:12 -0700 Subject: [PATCH 17/32] put null record dropping back into iceberg writer --- .../iceberg/connect/data/IcebergWriter.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java index e9cff67d..bc555944 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java @@ -59,13 +59,15 @@ private void initNewWriter() { public void write(SinkRecord record) { try { // TODO: config to handle tombstones instead of always ignoring? - Record row = convertToRow(record); - String cdcField = config.tablesCdcField(); - if (cdcField == null) { - writer.write(row); - } else { - Operation op = extractCdcOperation(record.value(), cdcField); - writer.write(new RecordWrapper(row, op)); + if (record.value() != null) { + Record row = convertToRow(record); + String cdcField = config.tablesCdcField(); + if (cdcField == null) { + writer.write(row); + } else { + Operation op = extractCdcOperation(record.value(), cdcField); + writer.write(new RecordWrapper(row, op)); + } } } catch (Exception e) { throw new DataException( From d89f15aa617dc8a9d9d410a63dbdf20a3364da17 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 26 Apr 2024 16:54:34 -0700 Subject: [PATCH 18/32] fix dead letter utils private constructor --- .../tabular/iceberg/connect/deadletter/DeadLetterUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index 30f0039c..eb905181 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -49,9 +49,7 @@ public Throwable getError() { } } - private DeadLetterUtils() { - throw new IllegalStateException("Should not be initialialized"); - } + private DeadLetterUtils() {} public static final String KEY_HEADER = "t_original_key"; public static final String VALUE_HEADER = "t_original_value"; From c5c218698f721498ac710dda55faeaee08da4e03 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 26 Apr 2024 16:59:16 -0700 Subject: [PATCH 19/32] fix cruft in readme --- README.md | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/README.md b/README.md index c5d94eb5..7ce5c79e 100644 --- a/README.md +++ b/README.md @@ -20,23 +20,6 @@ The zip archive will be found under `./kafka-connect-runtime/build/distributions # Configuration -iceberg.deadletter.enabled - -// this is now on the transform somewhere -// and MUST be supplied -// and is validated that FailedRecordFactory.schema provides it. -iceberg.deadletter.table_route_field // sure -// but also needs to be on the connector I think... I don't think the connector sees the transform settings <- LOOK INTO THIS - - -// iceberg.deadletter.table - -// iceberg.table.\
.partition-by <- we can add a partitioning scheme to it -// probably topic - -// connector that is handling 10 topics for customer A going to Catalog A - - | Property | Description | |--------------------------------------------|------------------------------------------------------------------------------------------------------------------| | iceberg.tables | Comma-separated list of destination tables | From 41c4372d52941e9c3f5dd424c00604589b607cf8 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 29 Apr 2024 19:28:04 -0700 Subject: [PATCH 20/32] post-merge fixes --- .../connect/transforms/ErrorTransform.java | 15 ++++--- .../iceberg/connect/channel/TaskImpl.java | 2 +- .../iceberg/connect/channel/Worker.java | 45 ++++++------------- .../iceberg/connect/channel/WorkerTest.java | 6 ++- 4 files changed, 28 insertions(+), 40 deletions(-) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index bb61bffe..710f0c53 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -54,13 +54,14 @@ * passed and namespaced * *

"transforms": "tab", "transforms.tab.type": - * "io.tabular.iceberg.connect.transform.managed.ManagedTransform", - * "transforms.tab.value.converter": "org.apache.kafka.connect.storage.StringConverter", - * "transforms.tab.value.converter.some_property: "...", "transforms.tab.key.converter": - * "org.apache.kafka.connect.storage.StringConverter", "transforms.tab.key.converter.some_property": - * "...", "transforms.tab.smts" : "some.java.class,some.other.java.class", - * "transforms.tab.smts.prop1" : "some_property_for_the_smts" - * + *

    + *
  • "io.tabular.iceberg.connect.transform.managed.ManagedTransform"
  • + *
  • "transforms.tab.value.converter": "org.apache.kafka.connect.storage.StringConverter"
  • + *
  • "transforms.tab.value.converter.some_property: "...", "transforms.tab.key.converter":
  • + *
  • "org.apache.kafka.connect.storage.StringConverter", "transforms.tab.key.converter.some_property": "..."
  • + *
  • "transforms.tab.smts" : "some.java.class,some.other.java.class"
  • + *
  • "transforms.tab.smts.prop1" : "some_property_for_the_smts"
  • + *
*

This should not be used with any other SMT. All SMTs should be added to "transforms.tab.smts". * *

It returns a special Map of String -> Object "original" : Map of String -> Object containing diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TaskImpl.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TaskImpl.java index f7b7e83d..ad9bec79 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TaskImpl.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/TaskImpl.java @@ -33,7 +33,7 @@ public class TaskImpl implements Task, AutoCloseable { public TaskImpl(SinkTaskContext context, IcebergSinkConfig config) { this.catalog = Utilities.loadCatalog(config); - this.writer = new Worker(config, catalog); + this.writer = new Worker(context, config, catalog); this.committer = new CommitterImpl(context, config, catalog); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 450ddef0..335bf2da 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -18,60 +18,49 @@ */ package io.tabular.iceberg.connect.channel; -import static java.util.stream.Collectors.toList; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.data.*; import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Map; + +import io.tabular.iceberg.connect.data.IcebergWriterFactory; +import io.tabular.iceberg.connect.data.Offset; +import io.tabular.iceberg.connect.data.RecordRouter; +import io.tabular.iceberg.connect.data.WriterManager; +import io.tabular.iceberg.connect.data.WriterResult; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.kafka.connect.sink.SinkTaskContext; // TODO: rename to WriterImpl later, minimize changes for clearer commit history for now class Worker implements Writer, AutoCloseable { - - private static final Logger LOG = LoggerFactory.getLogger(Worker.class); - private final IcebergSinkConfig config; - private final IcebergWriterFactory writerFactory; private final WriterManager writers; private final Map sourceOffsets; private final RecordRouter recordRouter; - Worker(IcebergSinkConfig config, Catalog catalog) { - this(config, new IcebergWriterFactory(catalog, config)); - this.writers = new WriterManager(writerFactory); - - this.recordRouter = - RecordRouter.from(writers, config, this.getClass().getClassLoader(), context); + Worker(SinkTaskContext context, IcebergSinkConfig config, Catalog catalog) { + this(context, config, new IcebergWriterFactory(catalog, config)); } @VisibleForTesting - Worker(IcebergSinkConfig config, IcebergWriterFactory writerFactory) { - this.config = config; - this.writerFactory = writerFactory; - this.writers = new WriterManager(writerFactory); + Worker(SinkTaskContext context, IcebergSinkConfig config, IcebergWriterFactory writerFactory) { + this.writers = new WriterManager(writerFactory); this.sourceOffsets = Maps.newHashMap(); - this.recordRouter = + this.recordRouter = RecordRouter.from(writers, config, this.getClass().getClassLoader(), context); } @Override public Committable committable() { - List writeResults = writers.writeResults(); - - List writeResults = - writers.values().stream().flatMap(writer -> writer.complete().stream()).collect(toList()); + List writeResults = writers.writeResults(); Map offsets = Maps.newHashMap(sourceOffsets); writers.clear(); @@ -82,7 +71,7 @@ public Committable committable() { @Override public void close() throws IOException { - writers.values().forEach(RecordWriter::close); + writers.stop(); writers.clear(); sourceOffsets.clear(); } @@ -103,10 +92,4 @@ private void save(SinkRecord record) { recordRouter.write(record); } - -// @Override -// public void stop() { -// super.stop(); -// writers.stop(); -// } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 78a627fd..fb0a9415 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -35,6 +35,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; import org.junit.jupiter.api.Test; public class WorkerTest { @@ -60,6 +61,9 @@ public void testDynamicRoute() { } private void workerTest(IcebergSinkConfig config, Map value) { + SinkTaskContext context = mock(SinkTaskContext.class); + + WriterResult writeResult = new WriterResult( TableIdentifier.parse(TABLE_NAME), @@ -72,7 +76,7 @@ private void workerTest(IcebergSinkConfig config, Map value) { IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); - Writer worker = new Worker(config, writerFactory); + Writer worker = new Worker(context, config, writerFactory); // save a record SinkRecord rec = new SinkRecord(SRC_TOPIC_NAME, 0, null, "key", null, value, 0L); From 13220e9557212150599e52f0b6294d68f22f480c Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 29 Apr 2024 19:33:00 -0700 Subject: [PATCH 21/32] more comments removing cruft --- kafka-connect-deadletter/build.gradle | 12 ------ .../connect/deadletter/DeadLetterUtils.java | 40 ------------------- 2 files changed, 52 deletions(-) diff --git a/kafka-connect-deadletter/build.gradle b/kafka-connect-deadletter/build.gradle index edfe399a..93ca7c16 100644 --- a/kafka-connect-deadletter/build.gradle +++ b/kafka-connect-deadletter/build.gradle @@ -6,19 +6,7 @@ dependencies { implementation libs.iceberg.core implementation libs.iceberg.common implementation libs.iceberg.guava - implementation libs.avro compileOnly libs.bundles.kafka.connect - - - testImplementation libs.junit.api - testRuntimeOnly libs.junit.engine - - testImplementation libs.mockito - testImplementation libs.assertj - - testFixturesImplementation libs.iceberg.common - testFixturesImplementation libs.iceberg.core - testFixturesImplementation libs.avro } publishing { diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index eb905181..394d9738 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -72,33 +72,6 @@ public static String stackTrace(Throwable error) { return sw.toString(); } - // public static class Values { - // // expect byte[] - // private final Object keyBytes; - // // expect byte[] - // private final Object valueBytes; - // // expect List - // private final Object headers; - // - // public Values(Object keyBytes, Object valueBytes, Object headers) { - // this.keyBytes = keyBytes; - // this.valueBytes = valueBytes; - // this.headers = headers; - // } - // - // public Object getKeyBytes() { - // return keyBytes; - // } - // - // public Object getValueBytes() { - // return valueBytes; - // } - // - // public Object getHeaders() { - // return headers; - // } - // } - /** * No way to get back the original Kafka header bytes. We instead have an array with elements of * {"key": String, "value": bytes} for each header. This can be converted back into a Kafka @@ -119,19 +92,6 @@ public static List serializedHeaders(SinkRecord original) { return headers; } - // @SuppressWarnings("unchecked") - // public static SinkRecord mapToFailedRecord( - // String targetTable, SinkRecord record, String location, Throwable error, String - // identifier) { - // Map payload = (Map) record.value(); - // Map bytes = (Map) payload.get(ORIGINAL_BYTES_KEY); - // Object keyBytes = bytes.get(KEY_BYTES); - // Object valueBytes = bytes.get(VALUE_BYTES); - // Object headers = bytes.get(HEADERS); - // Values values = new Values(keyBytes, valueBytes, headers); - // return failedRecord(record, values, error, location, targetTable, identifier); - // } - public static Object loadClass(String name, ClassLoader loader) { if (name == null || name.isEmpty()) { throw new IllegalArgumentException("cannot initialize empty class"); From 797b8612e86ee46819609921b20aa2bdf4917ea3 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 29 Apr 2024 19:36:09 -0700 Subject: [PATCH 22/32] regexrecordrouter --- .../java/io/tabular/iceberg/connect/data/RecordRouter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index f5a5c59e..18722ad5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -62,7 +62,7 @@ public static RecordRouter from( config.tables().forEach(TableIdentifier::of); baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); } else { - baseRecordRouter = new StaticRecordRouter(writers, config); + baseRecordRouter = new RegexRecordRouter(writers, config); } } @@ -101,12 +101,12 @@ public void write(SinkRecord record) { } } - private static class StaticRecordRouter extends RecordRouter { + private static class RegexRecordRouter extends RecordRouter { private final String routeField; private final WriterManager writers; private final IcebergSinkConfig config; - StaticRecordRouter(WriterManager writers, IcebergSinkConfig config) { + RegexRecordRouter(WriterManager writers, IcebergSinkConfig config) { this.routeField = config.tablesRouteField(); this.writers = writers; this.config = config; From 7bd7d5d973cf9a3893f79fe764fa9524e17ad247 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 30 Apr 2024 14:22:27 -0700 Subject: [PATCH 23/32] start of fallback mode --- .../iceberg/connect/data/RecordRouter.java | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 18722ad5..4abdd743 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -21,6 +21,7 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; + import java.util.List; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -60,8 +61,15 @@ public static RecordRouter from( // validate all table identifiers are valid, otherwise exception is thrown // as this is an invalid config setting, not an error during processing config.tables().forEach(TableIdentifier::of); - baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); + if (config.deadLetterTableEnabled()) { + + // need a config option to find this one + baseRecordRouter = new FallbackRecordRouter(new DynamicRecordRouter(writers, config.tablesRouteField()), new ConfigRecordRouter(writers, config.tables())); + } else{ + baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); + } } else { + // does this need a fallback or can it even have a fallback :thinking-face: baseRecordRouter = new RegexRecordRouter(writers, config); } } @@ -158,6 +166,24 @@ public void write(SinkRecord record) { } } + private static class FallbackRecordRouter extends RecordRouter { + private final RecordRouter primary; + private final RecordRouter fallback; + + FallbackRecordRouter(RecordRouter primary, RecordRouter fallback) { + this.primary = primary; + this.fallback = fallback; + } + + public void write(SinkRecord record) { + try { + primary.write(record); // this doesn't work because of the null. or rather test this out. + } catch (Exception error) { + fallback.write(record); + } + } + } + private static class ErrorHandlingRecordRouter extends RecordRouter { private final WriteExceptionHandler handler; private final WriterManager writers; From bff7233d169c137f1d3a50764c1ac2fbd5748df3 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 13 May 2024 14:04:14 -0700 Subject: [PATCH 24/32] third mode --- .../DefaultFailedRecordFactory.java | 35 ++- .../deadletter/FailedRecordFactory.java | 4 - .../iceberg/connect/IcebergSinkConfig.java | 4 +- .../data/DefaultWriteExceptionHandler.java | 15 +- .../iceberg/connect/data/RecordRouter.java | 133 +++++------ .../iceberg/connect/data/WriteException.java | 9 + .../connect/data/WriteExceptionHandler.java | 23 +- .../connect/data/RecordRouterTest.java | 207 ++++++++++++++++++ 8 files changed, 307 insertions(+), 123 deletions(-) create mode 100644 kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java index d76ed7c9..cd0c8810 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java @@ -29,8 +29,9 @@ import org.apache.kafka.connect.transforms.util.SimpleConfig; public class DefaultFailedRecordFactory implements FailedRecordFactory { - private static final String DEAD_LETTER_TABLE_NAME_PROP = "table_name"; + + private static final String DEAD_LETTER_ROUTE_FIELD_PROP = "route_field"; private static final ConfigDef CONFIG_DEF = new ConfigDef() .define( @@ -38,12 +39,18 @@ public class DefaultFailedRecordFactory implements FailedRecordFactory { ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "dead letter table name namespace.table"); + "dead letter table name namespace.table") + .define(DEAD_LETTER_ROUTE_FIELD_PROP, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + "route field to inject table name on"); private static final String HEADERS = "headers"; private Schema schema; private String deadLetterTableName; + private String deadLetterRouteField; @Override public Schema schema(String context) { @@ -100,29 +107,11 @@ public SinkRecord recordFromConnector(SinkRecord record, Throwable error, String record.topic(), record.kafkaPartition(), null, null, schema, struct, record.timestamp()); } - @Override - public boolean isFailedTransformRecord(SinkRecord record) { - if (record != null && record.valueSchema() != null) { - Map parameters = record.valueSchema().parameters(); - if (parameters != null) { - String isFailed = parameters.get("transform_failed"); - if (isFailed != null) { - return isFailed.equals("true"); - } - } - } - return false; - } - - @Override - public String tableName(SinkRecord record) { - return deadLetterTableName; - } - @Override public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); deadLetterTableName = config.getString(DEAD_LETTER_TABLE_NAME_PROP); + deadLetterRouteField = config.getString(DEAD_LETTER_ROUTE_FIELD_PROP); if (deadLetterTableName == null) { throw new IllegalArgumentException("Dead letter table name cannot be null"); } @@ -141,7 +130,8 @@ public void configure(Map props) { .field(HEADERS, DeadLetterUtils.HEADER_SCHEMA) .field("context", Schema.OPTIONAL_STRING_SCHEMA) .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) - .schema(); + .field(deadLetterRouteField, Schema.STRING_SCHEMA) + .build(); } private void addCommon(Struct struct, SinkRecord record, Throwable error, String context) { @@ -157,5 +147,6 @@ private void addCommon(Struct struct, SinkRecord record, Throwable error, String if (context != null) { struct.put("context", context); } + struct.put(deadLetterRouteField, deadLetterTableName); } } diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java index d0ff167b..db4c4ba0 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java @@ -32,9 +32,5 @@ public interface FailedRecordFactory { // where in the original are the byte arrays. SinkRecord recordFromConnector(SinkRecord record, Throwable error, String context); - boolean isFailedTransformRecord(SinkRecord record); - - String tableName(SinkRecord record); - void configure(Map props); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 3933cba7..77276ea9 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -88,8 +88,6 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.deadletter.record_factory"; private static final String FAILED_RECORD_FACTORY_PREFIX = "iceberg.tables.deadletter.record_factory"; - private static final String FAILED_RECORD_FACTORY_DEFAULT = - "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -250,7 +248,7 @@ private static ConfigDef newConfigDef() { configDef.define( FAILED_RECORD_FACTORY_PROP, Type.STRING, - FAILED_RECORD_FACTORY_DEFAULT, + null, Importance.MEDIUM, "If writing to Dead Letter Table, failed record factory class to use"); return configDef; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java index 4aca6140..c78b5535 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java @@ -34,7 +34,7 @@ public void initialize( } @Override - public Result handle(SinkRecord record, Exception exception) { + public SinkRecord handle(SinkRecord record, Exception exception) { if (exception instanceof WriteException) { return handleWriteException(record, (WriteException) exception); } @@ -46,7 +46,7 @@ public Result handle(SinkRecord record, Exception exception) { } @SuppressWarnings("checkstyle:CyclomaticComplexity") - private Result handleWriteException(SinkRecord record, WriteException exception) { + private SinkRecord handleWriteException(SinkRecord record, WriteException exception) { if (exception instanceof WriteException.CreateTableException) { Throwable cause = exception.getCause(); if (cause instanceof IllegalArgumentException || cause instanceof ValidationException) { @@ -81,14 +81,7 @@ private Result handleWriteException(SinkRecord record, WriteException exception) throw exception; } - private Result failedRecord(SinkRecord record, WriteException exception) { - String targetTableId = exception.tableId(); - if (targetTableId != null && targetTableId.equals(factory.tableName(record))) { - throw new IllegalArgumentException( - String.format( - "Must throw for exceptions involving target Dead Letter Table: %s", targetTableId), - exception); - } - return new Result(factory.recordFromConnector(record, exception, null), targetTableId); + private SinkRecord failedRecord(SinkRecord record, WriteException exception) { + return factory.recordFromConnector(record, exception, null); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 4abdd743..5cd429c1 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -42,34 +42,55 @@ protected final String extractRouteValue(Object recordValue, String routeField) } catch (Exception error) { throw new WriteException.RouteException(error); } + + // confirm if this can even happen return routeValue == null ? null : routeValue.toString(); } + + /* + |iceberg.tables|dynamic-enables|route-field| routing behavior | + |--------------|---------------|----------------|-------------| + | empty | true | populated | DynamicRecordRouter | + | empty | false | populated | RegexRouter | + | populated | false | null | ConfigRouter | + | populated | false | populated | DynamicRecordRouter then ConfigRouter | + + what does iceberg.tables.default-commit-branch do ? + + Record routing is complex due to maintaining non-breaking config changes +

    +
  • if iceberg.tables.dynamic-enabled is true then we route based on iceberg.tables.route-field, regardless of other fields +
  • if iceberg.tables.dynamic-enabled is false and iceberg.tables is empty, we use routeRegex +
  • if iceberg.tables.dynamic-enabled is false and iceberg.tables is populated and route-field is empty, we route to all listed tables +
  • as above, but if route-field is set we attempt to route dynamically and if that field does not exist we do the behavior above. +
+

+ The last option is required for Dead Letter Table handling while routing to iceberg.tables, as the dead letter table routing + is similar to dynamic routing: based on a field and under the control of the user.

+ + How to avoid the infinite loop problem? + */ public static RecordRouter from( - WriterManager writers, - IcebergSinkConfig config, - ClassLoader loader, - SinkTaskContext context) { + WriterManager writers, + IcebergSinkConfig config, + ClassLoader loader, + SinkTaskContext context) { RecordRouter baseRecordRouter; if (config.dynamicTablesEnabled()) { Preconditions.checkNotNull( - config.tablesRouteField(), "Route field cannot be null with dynamic routing"); + config.tablesRouteField(), "Route field cannot be null with dynamic routing"); baseRecordRouter = new DynamicRecordRouter(writers, config.tablesRouteField()); } else { - if (config.tablesRouteField() == null) { - // validate all table identifiers are valid, otherwise exception is thrown - // as this is an invalid config setting, not an error during processing + if (config.tables() != null && !config.tables().isEmpty()) { config.tables().forEach(TableIdentifier::of); - if (config.deadLetterTableEnabled()) { - - // need a config option to find this one + if (config.tablesRouteField() != null) { baseRecordRouter = new FallbackRecordRouter(new DynamicRecordRouter(writers, config.tablesRouteField()), new ConfigRecordRouter(writers, config.tables())); - } else{ + } else { baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); } } else { - // does this need a fallback or can it even have a fallback :thinking-face: baseRecordRouter = new RegexRecordRouter(writers, config); } } @@ -78,19 +99,19 @@ public static RecordRouter from( String failedRecordFactoryClass = config.getFailedRecordHandler(); String handlerClass = config.getWriteExceptionHandler(); FailedRecordFactory factory = - (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, loader); + (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, loader); factory.configure(config.failedRecordHandlerProperties()); WriteExceptionHandler handler = - (WriteExceptionHandler) DeadLetterUtils.loadClass(handlerClass, loader); + (WriteExceptionHandler) DeadLetterUtils.loadClass(handlerClass, loader); handler.initialize(context, config, factory); baseRecordRouter = - new RecordRouter.ErrorHandlingRecordRouter(baseRecordRouter, handler, writers, factory); + new RecordRouter.ErrorHandlingRecordRouter(baseRecordRouter, handler); } return baseRecordRouter; } - private static class ConfigRecordRouter extends RecordRouter { + public static class ConfigRecordRouter extends RecordRouter { private final List tables; private final WriterManager writers; @@ -103,13 +124,13 @@ private static class ConfigRecordRouter extends RecordRouter { public void write(SinkRecord record) { // route to all tables tables.forEach( - tableName -> { - writers.write(tableName, record, false); - }); + tableName -> { + writers.write(tableName, record, false); + }); } } - private static class RegexRecordRouter extends RecordRouter { + public static class RegexRecordRouter extends RecordRouter { private final String routeField; private final WriterManager writers; private final IcebergSinkConfig config; @@ -125,29 +146,29 @@ public void write(SinkRecord record) { String routeValue = extractRouteValue(record.value(), routeField); if (routeValue != null) { config - .tables() - .forEach( - tableName -> - config - .tableConfig(tableName) - .routeRegex() - .ifPresent( - regex -> { - boolean matches; - try { - matches = regex.matcher(routeValue).matches(); - } catch (Exception error) { - throw new WriteException.RouteRegexException(error); - } - if (matches) { - writers.write(tableName, record, false); - } - })); + .tables() + .forEach( + tableName -> + config + .tableConfig(tableName) + .routeRegex() + .ifPresent( + regex -> { + boolean matches; + try { + matches = regex.matcher(routeValue).matches(); + } catch (Exception error) { + throw new WriteException.RouteRegexException(error); + } + if (matches) { + writers.write(tableName, record, false); + } + })); } } } - private static class DynamicRecordRouter extends RecordRouter { + public static class DynamicRecordRouter extends RecordRouter { private final String routeField; private final WriterManager writers; @@ -166,7 +187,7 @@ public void write(SinkRecord record) { } } - private static class FallbackRecordRouter extends RecordRouter { + public static class FallbackRecordRouter extends RecordRouter { private final RecordRouter primary; private final RecordRouter fallback; @@ -177,43 +198,31 @@ private static class FallbackRecordRouter extends RecordRouter { public void write(SinkRecord record) { try { - primary.write(record); // this doesn't work because of the null. or rather test this out. + primary.write(record); } catch (Exception error) { fallback.write(record); } } } - private static class ErrorHandlingRecordRouter extends RecordRouter { + public static class ErrorHandlingRecordRouter extends RecordRouter { private final WriteExceptionHandler handler; - private final WriterManager writers; private final RecordRouter router; - private final FailedRecordFactory failedRecordFactory; ErrorHandlingRecordRouter( - RecordRouter baseRouter, - WriteExceptionHandler handler, - WriterManager writers, - FailedRecordFactory factory) { + RecordRouter baseRouter, + WriteExceptionHandler handler) { this.router = baseRouter; this.handler = handler; - this.writers = writers; - this.failedRecordFactory = factory; } @Override public void write(SinkRecord record) { - if (failedRecordFactory.isFailedTransformRecord(record)) { - writers.write(failedRecordFactory.tableName(record), record, false); - } else { - try { - router.write(record); - } catch (Exception error) { - WriteExceptionHandler.Result result = handler.handle(record, error); - if (result != null) { - writers.write(result.tableName(), result.sinkRecord(), false); - } - } + try { + router.write(record); + } catch (Exception error) { + SinkRecord result = handler.handle(record, error); + router.write(result); } } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java index 1cc41add..b1b2123b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java @@ -29,6 +29,11 @@ public class WriteException extends RuntimeException { tableIdentifier = null; } + WriteException(String msg) { + super(msg); + tableIdentifier = null; + } + WriteException(TableIdentifier tableId, Throwable cause) { super(cause); this.tableIdentifier = tableId.toString(); @@ -80,6 +85,10 @@ public static class RouteException extends WriteException { RouteException(Throwable cause) { super(cause); } + + RouteException(String msg) { + super(msg); + } } public static class RouteRegexException extends WriteException { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java index d2dd4cdc..d6cb12f4 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java @@ -26,35 +26,16 @@ public interface WriteExceptionHandler { void initialize(SinkTaskContext context, IcebergSinkConfig config, FailedRecordFactory factory); - class Result { - private final SinkRecord sinkRecord; - private final String tableName; - - public Result(SinkRecord sinkRecord, String tableName) { - this.sinkRecord = sinkRecord; - this.tableName = tableName; - } - - public SinkRecord sinkRecord() { - return sinkRecord; - } - - public String tableName() { - return tableName; - } - } - /** * This method will be invoked whenever the connector runs into an exception while trying to write * SinkRecords to a table. Implementations of this method have 3 general options: * *
    - *
  1. Return a SinkRecord and the name of the table to write to (wrapped inside a {@link - * Result}) + *
  2. Return a SinkRecord *
  3. Return null to drop the SinkRecord *
* * @param record The SinkRecord that couldn't be written */ - Result handle(SinkRecord record, Exception exception); + SinkRecord handle(SinkRecord record, Exception exception); } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java new file mode 100644 index 00000000..27a5da24 --- /dev/null +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.tabular.iceberg.connect.data; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Pair; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.stream.Collectors; + +public class RecordRouterTest { + + static class RecordingWriterManager extends WriterManager { + + private final List> log; + RecordingWriterManager(IcebergWriterFactory factory) { + super(factory); + this.log = Lists.newArrayList(); + } + + @Override + public void write(String tableName, SinkRecord record, boolean ignoreMissingTable) { + log.add(Pair.of(tableName, record)); + } + + } + + private static final IcebergWriterFactory factory = mock(IcebergWriterFactory.class); + private static final SinkTaskContext context = mock(SinkTaskContext.class); + + + @Test + @DisplayName("ConfigRouter should dispatch based on configured tables") + public void configRouterTest() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + SinkRecord record = mock(SinkRecord.class); + + when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); + when(config.deadLetterTableEnabled()).thenReturn(false); + when(config.tablesRouteField()).thenReturn(null); + when(config.dynamicTablesEnabled()).thenReturn(false); + + RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); + // do some assertions here. + assertThat(router).isInstanceOf(RecordRouter.ConfigRecordRouter.class); + // test some dispatching here. + router.write(record); + List> result = manager.log; + assertThat(result).isEqualTo(Lists.newArrayList(Pair.of("tbl1", record), Pair.of("tbl2", record))); + } + + @Test + @DisplayName("Fallback writer should dispatch based on record value and fall back to configured tables otherwise") + public void fallBackWriterTest() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + + Schema schemaWithRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA).field("route_field", Schema.STRING_SCHEMA).build(); + Schema schemaWithoutRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA); + + Struct structWithRoute = new Struct(schemaWithRoute).put("a", "a").put("route_field", "route_field_table"); + Struct structWithoutRoute = new Struct(schemaWithoutRoute).put("a", "a"); + + SinkRecord recordWithRoute = new SinkRecord("topic", 1, null, null, schemaWithRoute, structWithRoute, 100L); + SinkRecord recordWithoutRoute = new SinkRecord("topic", 1, null, null, schemaWithoutRoute, structWithoutRoute, 101L); + + when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); + when(config.deadLetterTableEnabled()).thenReturn(false); + when(config.tablesRouteField()).thenReturn("route_field"); + when(config.dynamicTablesEnabled()).thenReturn(false); + + RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); + // do some assertions here. + assertThat(router).isInstanceOf(RecordRouter.FallbackRecordRouter.class); + // test some dispatching here. + router.write(recordWithRoute); + router.write(recordWithoutRoute); + List> result = manager.log; + assertThat(result).isEqualTo(Lists.newArrayList(Pair.of("route_field_table", recordWithRoute), Pair.of("tbl1", recordWithoutRoute), Pair.of("tbl2", recordWithoutRoute))); + } + + @Test + @DisplayName("DynamicRecordRouter should dispatch based on the record field") + public void dynamicRecordRouterTest() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + + Schema schemaWithRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA).field("route_field", Schema.STRING_SCHEMA).build(); + Schema schemaWithoutRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA); + + Struct structWithRoute = new Struct(schemaWithRoute).put("a", "a").put("route_field", "route_field_table"); + Struct structWithoutRoute = new Struct(schemaWithoutRoute).put("a", "a"); + + SinkRecord recordWithRoute = new SinkRecord("topic", 1, null, null, schemaWithRoute, structWithRoute, 100L); + SinkRecord recordWithoutRoute = new SinkRecord("topic", 1, null, null, schemaWithoutRoute, structWithoutRoute, 101L); + + when(config.tables()).thenReturn(Lists.newArrayList()); + when(config.deadLetterTableEnabled()).thenReturn(false); + when(config.tablesRouteField()).thenReturn("route_field"); + when(config.dynamicTablesEnabled()).thenReturn(true); + + RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); + assertThat(router).isInstanceOf(RecordRouter.DynamicRecordRouter.class); + + router.write(recordWithRoute); + List> result = manager.log; + + assertThat(result).isEqualTo(Lists.newArrayList(Pair.of("route_field_table", recordWithRoute))); + assertThrows(WriteException.RouteException.class, () -> router.write(recordWithoutRoute)); + } + + @Test + @DisplayName("RegexRouter should be configured when dynamicTablesEnabled is false and iceberg.tables is null or empty") + public void regexRouterTest() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + + IcebergSinkConfig configTablesNull = mock(IcebergSinkConfig.class); + when(configTablesNull.tables()).thenReturn(null); + when(configTablesNull.deadLetterTableEnabled()).thenReturn(false); + when(configTablesNull.tablesRouteField()).thenReturn("route_val"); + when(configTablesNull.dynamicTablesEnabled()).thenReturn(false); + + IcebergSinkConfig configTablesEmpty = mock(IcebergSinkConfig.class); + when(configTablesEmpty.tables()).thenReturn(Lists.newArrayList()); + when(configTablesEmpty.deadLetterTableEnabled()).thenReturn(false); + when(configTablesEmpty.tablesRouteField()).thenReturn("route_val"); + when(configTablesEmpty.dynamicTablesEnabled()).thenReturn(false); + + RecordRouter routerNull = RecordRouter.from(manager, configTablesNull, this.getClass().getClassLoader(), context); + RecordRouter routerEmpty = RecordRouter.from(manager, configTablesEmpty, this.getClass().getClassLoader(), context); + + assertThat(routerNull).isInstanceOf(RecordRouter.RegexRecordRouter.class); + assertThat(routerEmpty).isInstanceOf(RecordRouter.RegexRecordRouter.class); + } + + @Test + @DisplayName("ErrorHandlingRouter should be configured when deadLetterTableEnabled is true") + public void errorHandlingRouterGetsConfiguredProperly() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + + Schema schemaWithRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA).field("route_field", Schema.STRING_SCHEMA).build(); + Schema schemaWithoutRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA); + + Struct structWithRoute = new Struct(schemaWithRoute).put("a", "a").put("route_field", "route_field_table"); + Struct structWithoutRoute = new Struct(schemaWithoutRoute).put("a", "bad_record_fail"); + + SinkRecord recordWithRoute = new SinkRecord("topic", 1, null, null, schemaWithRoute, structWithRoute, 100L); + SinkRecord recordWithoutRoute = new SinkRecord("topic", 1, null, null, schemaWithoutRoute, structWithoutRoute, 101L); + + // defaultRecordFactory assumes ErrorTransform has been used to put original bytes on the records in the headers + // since this record will fail, it will go through the configured DefaultFailedRecordFactory and expects these values to be present + recordWithoutRoute.headers().add(DeadLetterUtils.VALUE_HEADER, new SchemaAndValue(SchemaBuilder.OPTIONAL_BYTES_SCHEMA, "test".getBytes(StandardCharsets.UTF_8))); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn("route_field"); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.getFailedRecordHandler()).thenReturn("io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"); + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DefaultWriteExceptionHandler"); + when(config.failedRecordHandlerProperties()).thenReturn(ImmutableMap.of("table_name", "dlt.table", "route_field", "route_field")); + + RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); + assertThat(router).isInstanceOf(RecordRouter.ErrorHandlingRecordRouter.class); + + router.write(recordWithRoute); + router.write(recordWithoutRoute); + + List> result = manager.log; + assertThat(result.stream().map(Pair::first).collect(Collectors.toList())).isEqualTo(Lists.newArrayList("route_field_table", "dlt.table")); + + } + +} From 25208da8046aac7696199e01a32d9b3a6354c3ce Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 13 May 2024 14:43:21 -0700 Subject: [PATCH 25/32] another test case --- .../connect/data/RecordRouterTest.java | 36 +++++++++++++++++-- 1 file changed, 33 insertions(+), 3 deletions(-) diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java index 27a5da24..2b1787fc 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -103,9 +103,7 @@ public void fallBackWriterTest() { when(config.dynamicTablesEnabled()).thenReturn(false); RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); - // do some assertions here. assertThat(router).isInstanceOf(RecordRouter.FallbackRecordRouter.class); - // test some dispatching here. router.write(recordWithRoute); router.write(recordWithoutRoute); List> result = manager.log; @@ -201,7 +199,39 @@ public void errorHandlingRouterGetsConfiguredProperly() { List> result = manager.log; assertThat(result.stream().map(Pair::first).collect(Collectors.toList())).isEqualTo(Lists.newArrayList("route_field_table", "dlt.table")); - } + @Test + @DisplayName("ErrorHandlingRouter should throw if there is an issue with the failed record conversion") + public void errorHandlingRouterDoesNotInfiniteLoop() { + RecordingWriterManager manager = new RecordingWriterManager(factory); + + Schema schemaWithoutRoute = SchemaBuilder.struct().field("a", Schema.STRING_SCHEMA); + + Struct structWithoutRoute = new Struct(schemaWithoutRoute).put("a", "bad_record_fail"); + + SinkRecord recordWithoutRoute = new SinkRecord("topic", 1, null, null, schemaWithoutRoute, structWithoutRoute, 101L); + + // defaultRecordFactory assumes ErrorTransform has been used to put original bytes on the records in the headers + // since this record will fail, it will go through the configured DefaultFailedRecordFactory and expects these values to be present + recordWithoutRoute.headers().add(DeadLetterUtils.VALUE_HEADER, new SchemaAndValue(SchemaBuilder.OPTIONAL_BYTES_SCHEMA, "test".getBytes(StandardCharsets.UTF_8))); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); + when(config.deadLetterTableEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn("route_field"); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.getFailedRecordHandler()).thenReturn("io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"); + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DefaultWriteExceptionHandler"); + // the underlying router is looking for `route_field` but the failed record handler is configured to have + // the route field on `route_field_bad` + // this should cause the ErrorHandler to throw an exception + // since this is a configuration issue, it should kill the connector w/ unhandled exception + when(config.failedRecordHandlerProperties()).thenReturn(ImmutableMap.of("table_name", "dlt.table", "route_field", "route_field_bad")); + + RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); + assertThat(router).isInstanceOf(RecordRouter.ErrorHandlingRecordRouter.class); + + assertThrows(WriteException.RouteException.class, () -> router.write(recordWithoutRoute)); + } } From 205c2d7a699dc8654e07600fde88aa53780b1cfd Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 13 May 2024 16:10:34 -0700 Subject: [PATCH 26/32] better regex detection to avoid an extra config --- .../iceberg/connect/data/RecordRouter.java | 24 ++++++++++++++++++- .../connect/data/RecordRouterTest.java | 1 + 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 5cd429c1..5dece5f5 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -23,6 +23,7 @@ import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; import java.util.List; + import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.kafka.connect.sink.SinkRecord; @@ -86,7 +87,11 @@ public static RecordRouter from( if (config.tables() != null && !config.tables().isEmpty()) { config.tables().forEach(TableIdentifier::of); if (config.tablesRouteField() != null) { - baseRecordRouter = new FallbackRecordRouter(new DynamicRecordRouter(writers, config.tablesRouteField()), new ConfigRecordRouter(writers, config.tables())); + if (hasRegexMode(config)) { + baseRecordRouter = new RegexRecordRouter(writers, config); + } else { + baseRecordRouter = new FallbackRecordRouter(new DynamicRecordRouter(writers, config.tablesRouteField()), new ConfigRecordRouter(writers, config.tables())); + } } else { baseRecordRouter = new ConfigRecordRouter(writers, config.tables()); } @@ -111,6 +116,23 @@ public static RecordRouter from( return baseRecordRouter; } + private static boolean hasRegexMode(IcebergSinkConfig config) { + long definedRegexes = config + .tables() + .stream() + .map( + tableName -> { + try { +return config + .tableConfig(tableName) + .routeRegex().isPresent(); + } catch (Exception unused) { + return false; + } + }).filter(present -> present).count(); + return definedRegexes > 0; + } + public static class ConfigRecordRouter extends RecordRouter { private final List tables; private final WriterManager writers; diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java index 2b1787fc..7811ab7d 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -110,6 +110,7 @@ public void fallBackWriterTest() { assertThat(result).isEqualTo(Lists.newArrayList(Pair.of("route_field_table", recordWithRoute), Pair.of("tbl1", recordWithoutRoute), Pair.of("tbl2", recordWithoutRoute))); } + @Test @DisplayName("DynamicRecordRouter should dispatch based on the record field") public void dynamicRecordRouterTest() { From 4aeedcd35be3a198b0c6e5740e3fadd49f509dda Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Mon, 13 May 2024 16:27:56 -0700 Subject: [PATCH 27/32] cruft cleanup and starting docs --- README.md | 69 +++++++++++++++++++ .../iceberg/connect/data/RecordRouter.java | 25 ------- 2 files changed, 69 insertions(+), 25 deletions(-) diff --git a/README.md b/README.md index 7ce5c79e..a1d9e2a3 100644 --- a/README.md +++ b/README.md @@ -50,6 +50,9 @@ The zip archive will be found under `./kafka-connect-runtime/build/distributions | iceberg.hadoop-conf-dir | If specified, Hadoop config files in this directory will be loaded | | iceberg.hadoop.* | Properties passed through to the Hadoop configuration | | iceberg.kafka.* | Properties passed through to control topic Kafka client initialization | +| iceberg.tables.deadletter.handler | See Dead Letter Table Mode | +| iceberg.tables.deadletter.record_factpry | See Dead Letter Table Mode | +| iceberg.tables.deadletter.record_factory.* | see Dead Letter Table Mode | If `iceberg.tables.dynamic-enabled` is `false` (the default) then you must specify `iceberg.tables`. If `iceberg.tables.dynamic-enabled` is `true` then you must specify `iceberg.tables.route-field` which will @@ -322,6 +325,72 @@ See above for creating the table } ``` +## Dead Letter Table + +The connector can be configured to write to one or more Dead Letter iceberg tables, with capability beyond +what is offered from Kafka Connects Dead Letter Queue implementation. + +| Location of Failure | Kafka Connect DLQ | Dead Letter Table Mode | +|----------------------------------------------------------|-------------------|------------------------| +| Deserialization/Converter | Yes | Yes* | +| SMT | Yes | Yes* | +| Table creation / schema issues | No | Yes | +| Iceberg record conversion | No | Yes | +| Malformed records (e.g. missing table route information) | No | Yes | +| Schema evolution issues | No | Yes | + +If the `ErrorTransform` SMT is not used, it may be challenging to put records into the Dead Letter Table other than +recording metadata (Topic, Partition, Offset) while dropping the message. + +If the `ErrorTransform` SMT is used, failures can include the original bytes of the message in the Iceberg Table +that can be extracted/inspected using a downstream query engine. + +In order to use the ErrorTransform: + +... todo + +In order to turn on Dead Letter Table mode: + +... todo + +### Routing + +Dead Letter Table routing is a variation on Dynamic Routing --that is, a route field can be added by the +FailedRecordHandler that can be used to dispatch to one or more Dead Letter tables. + +| iceberg.tables | dynamic-enabled | route-field | routing behavior | +|----------------|-----------------|--------------|--------------------------------------------------------------| +| empty | true | populated | DynamicRecordRouter | +| empty | false | populated | RegexRouter | +| populated | false | populated | RegexRouter if iceberg.table.\
.route-regex set | +| populated | false | null | ConfigRouter | +| populated | false | populated | DynamicRecordRouter then ConfigRouter | + +### Partitioning + +The following properties still apply to the Dead Letter Table. The partition-by field can be used to customize the +partitioning of the Dead Letter table(s). + +| Property | Description | +|--------------------------------------------|------------------------------------------------------------------------------------------------| +| iceberg.table.\
.commit-branch | Table-specific branch for commits, use `iceberg.tables.default-commit-branch` if not specified | +| iceberg.table.\
.id-columns | Comma-separated list of columns that identify a row in the table (primary key) | +| iceberg.table.\
.partition-by | Comma-separated list of partition fields to use when creating the table | + +### ErrorTransform specifics + +... todo + +### WriteExceptionHandler Contract + +... todo + +### FailedRecordHandler Contract + +... todo + +### Provided Handlers + ## Resources * [Running IcebergSinkConnector locally](https://github.com/wuerike/kafka-iceberg-streaming) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 5dece5f5..0b470819 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -44,34 +44,9 @@ protected final String extractRouteValue(Object recordValue, String routeField) throw new WriteException.RouteException(error); } - // confirm if this can even happen return routeValue == null ? null : routeValue.toString(); } - - /* - |iceberg.tables|dynamic-enables|route-field| routing behavior | - |--------------|---------------|----------------|-------------| - | empty | true | populated | DynamicRecordRouter | - | empty | false | populated | RegexRouter | - | populated | false | null | ConfigRouter | - | populated | false | populated | DynamicRecordRouter then ConfigRouter | - - what does iceberg.tables.default-commit-branch do ? - - Record routing is complex due to maintaining non-breaking config changes -
    -
  • if iceberg.tables.dynamic-enabled is true then we route based on iceberg.tables.route-field, regardless of other fields -
  • if iceberg.tables.dynamic-enabled is false and iceberg.tables is empty, we use routeRegex -
  • if iceberg.tables.dynamic-enabled is false and iceberg.tables is populated and route-field is empty, we route to all listed tables -
  • as above, but if route-field is set we attempt to route dynamically and if that field does not exist we do the behavior above. -
-

- The last option is required for Dead Letter Table handling while routing to iceberg.tables, as the dead letter table routing - is similar to dynamic routing: based on a field and under the control of the user.

- - How to avoid the infinite loop problem? - */ public static RecordRouter from( WriterManager writers, IcebergSinkConfig config, From 0cf18d189b4fb48abfee4d8cfc4b2b6872117391 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 14 May 2024 08:47:21 -0700 Subject: [PATCH 28/32] fix error transform tests --- .../transforms/ErrorTransformTest.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java index e284aba3..325c79bc 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -57,6 +57,10 @@ public class ErrorTransformTest { private static final String DEAD_LETTER_TABLE_NAME = "dead_letter.table"; + private static final String DEAD_LETTER_ROUTE_FIELD_PROP = "route_field"; + + private static final String DEAD_LETTER_ROUTE_FIELD = "some_field"; + private final FailedRecordFactory failedRecordFactory = getFailedRecordFactory(); private Headers stringAsByteHeaders() { @@ -89,15 +93,10 @@ private FailedRecordFactory getFailedRecordFactory() { DeadLetterUtils.loadClass( "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory", this.getClass().getClassLoader()); - factory.configure(ImmutableMap.of("table_name", DEAD_LETTER_TABLE_NAME)); + factory.configure(ImmutableMap.of("table_name", DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); return factory; } - // private assertHeaders(Headers headers, String valueBytes, String keyBytes, List - // originalHeaders) { - // headers.lastWithName("t_") - // } - @Test @DisplayName( "It should deserialize using the supplied converters into the custom SinkRecord shape with additional headers") @@ -114,7 +113,7 @@ public void deserialize() { "header.converter.converter.type", "header", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders())); @@ -157,7 +156,7 @@ public void smt() { "smts.transform_text", transformString, "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); @@ -194,7 +193,7 @@ public void tombstone() { "key.converter", STRING_CONVERTER, "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord record = createRecord(null, null, null); assertThat(smt.apply(record)).isSameAs(record); } @@ -220,7 +219,7 @@ public void nullFilteredBySMT() { "smts.null", "true", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord result = smt.apply(createRecord(KEY_STRING, VALUE_STRING, null)); @@ -243,7 +242,7 @@ public void keyFailed() { "header.converter.converter.type", "header", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); String malformedKey = "{\"malformed_json\"\"\"{}{}{}{}**"; SinkRecord result = @@ -288,7 +287,7 @@ public void valueFailed() { "header.converter.converter.type", "header", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); String malformedValue = "{\"malformed_json\"\"\"{}{}{}{}**"; SinkRecord result = @@ -341,7 +340,7 @@ public void headerFailed() { "header.converter.converter.type", "header", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, headers); SinkRecord result = smt.apply(record); @@ -389,7 +388,7 @@ public void smtFailed() { "smts.throw", "true", "table_name", - DEAD_LETTER_TABLE_NAME)); + DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders()); SinkRecord result = smt.apply(record); From 05ea87f9bca6c43c832b3afc16e1dd05b89fb4fb Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 14 May 2024 13:53:08 -0700 Subject: [PATCH 29/32] more docs --- README.md | 82 ++++++++++++++----- .../transforms/DefaultExceptionHandler.java | 25 +++++- .../connect/transforms/ErrorTransform.java | 29 +++++-- .../transforms/TransformExceptionHandler.java | 2 +- .../iceberg/connect/IcebergSinkConfig.java | 6 +- 5 files changed, 112 insertions(+), 32 deletions(-) diff --git a/README.md b/README.md index a1d9e2a3..a178c231 100644 --- a/README.md +++ b/README.md @@ -328,7 +328,9 @@ See above for creating the table ## Dead Letter Table The connector can be configured to write to one or more Dead Letter iceberg tables, with capability beyond -what is offered from Kafka Connects Dead Letter Queue implementation. +what is offered from Kafka Connects Dead Letter Queue implementation. This is an optional setting you can +ignore if you want failures to cause the connector to fail. Alternatively, the Dead Letter Table mode can +be used to ignore all failures, beyond `error.tolerance = all` | Location of Failure | Kafka Connect DLQ | Dead Letter Table Mode | |----------------------------------------------------------|-------------------|------------------------| @@ -343,15 +345,64 @@ If the `ErrorTransform` SMT is not used, it may be challenging to put records in recording metadata (Topic, Partition, Offset) while dropping the message. If the `ErrorTransform` SMT is used, failures can include the original bytes of the message in the Iceberg Table -that can be extracted/inspected using a downstream query engine. +that can be extracted/inspected using a downstream query engine. -In order to use the ErrorTransform: +There are several classes that can be implemented and passed to the config for user customization: -... todo +* Error Transform SMT Key, Value, and Header deserialization failure handlers (each can be a different class) +* SMT transformation failure handlers +* Connector WriteException handler to handle issues with records themselves (e.g. missing route columns, invalid schema evolutions, etc.) +* Dead Letter Table schema / Record conversion -In order to turn on Dead Letter Table mode: +Some default implementations are provided. -... todo +NOTE: **Avro/Schema Registry** should not be used in conjunction with Dead Letter Table using the provided handlers. Avro deserialization +failures mix transient and non-transient errors in the same exception. A failure handler for avro deserialization is planned, but not +yet implemented. + +Users can extend these handlers to suit the needs of their particular deserialization method, catalog, etc. all of whihc can raise +different errors. It is advised to carefully develop these and start conservatively: you do not want to send messages to the Dead Letter +Table for transient network errors, which may be catalog specific. + +In order to use the ErrorTransform SMT: + +You **MUST** set the following due to how Kafka Connect displays keys to SMTs: +- `errors.tolerance` to `none` +- `key.converter` `value.converter`, and `header.converter` to `org.apache.kafka.connect.converters.ByteArrayConverter` +- set a `transforms` key. For example, you may set `transforms` : `error` , in which case all further configures are under `transforms.error.*` + +| Property | Description | +|------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------| +| transforms.error.type | **only** io.tabular.iceberg.connect.transforms.ErrorTransform , other SMTs will be defined elsewhere | +| transforms.error.value.converter | New location for the value converter | +| transforms.error.value.converter.* | properties to be passed to the value converter | +| transforms.error.key.converter | New location for the key converter | +| transforms.error.key.converter.* | properties to be passed to the key converter | +| transforms.error.header.converter | New location for the header converter | +| transforms.error.header.converter.* | properties to be passed to the header converter | +| transforms.error.smts | (Optional String): comma separated list of classes for additional SMTs to apply to the record after deserialization | +| transforms.error.smts.* | (Optional) properties to pass to the additional SMTS | +| transforms.error.converter | Class of io.tabular.iceberg.connect.transforms.TransformExceptionHandler to handle exceptions thrown by the key/value/header converter | +| transforms.error.smt | Class of io.tabular.iceberg.connect.transforms.TransformExceptionHandler to handle exceptions thrown the SMT chain | +| transforms.error.failed_record_factory | (Optional, property of the default exception handler): Class of io.tabular.iceberg.connect.deadletter.FailedRecordFactory | +| transforms.error.failed_record_factory.* | (Optional, property of the default exception handler): properties for the default exception handler | + +`io.tabular.iceberg.connect.transforms.DefaultExceptionHandler` is provided for simple use cases/reference. The handle method must return a `SinkRecord`. + +To maintain schema, record transformation parity between the SMT and the connector failures it is recommended to dynamically load a subclass of `io.tabular.iceberg.connect.deadletter.FailedRecordFactory`. +The reference implementation of `io.tabular.iceberg.connect.transforms.DefaultExceptionHandler` loads this by default. It can be configured on the `transforms.error.failed_record_factory` key. + +In order to turn on Dead Letter Table mode in the connector: + +| Property | Description | +|---------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------| +| iceberg.deadletter.handler | Sucblass of io.tabular.iceberg.connect.data.WriteExceptionHandler , if this is not-null Dead Letter Table mode is turned on in the connector | +| iceberg.deadletter.failed_record_factory | Scubclass of io.tabular.iceberg.connect.deadletter.FailedRecordFactory for converting records. The connector cannot see the SMT version | +| iceberg.deadletter.failed_record_factory.* | properties to be passed to the failed record factory | + +You do not need to use the Error SMT to turn on dead letter mode; however, the provided `io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory` assumes the Error SMT has been used +and will throw exceptions if not. You can implement your own WriteExceptionHandler/FailedRecordFactory to skip messages, transform messages, strip fields from messages and only write the +Kafka metadata etc. ### Routing @@ -366,6 +417,11 @@ FailedRecordHandler that can be used to dispatch to one or more Dead Letter tabl | populated | false | null | ConfigRouter | | populated | false | populated | DynamicRecordRouter then ConfigRouter | +Regardless of the above, if a WriteExceptionHandler `io.tabular.iceberg.connect.data.WriteExceptionHandler` is not null, Dead Letter Table mode +will wrap one of the underlying record routing modes. All exceptions are passed to the WriteExceptionHandler where +they can be ignored (message dropped), converted into a record and dispatched to the Dead Letter Table(s), or rethrown +to fail the connector. + ### Partitioning The following properties still apply to the Dead Letter Table. The partition-by field can be used to customize the @@ -377,20 +433,6 @@ partitioning of the Dead Letter table(s). | iceberg.table.\
.id-columns | Comma-separated list of columns that identify a row in the table (primary key) | | iceberg.table.\
.partition-by | Comma-separated list of partition fields to use when creating the table | -### ErrorTransform specifics - -... todo - -### WriteExceptionHandler Contract - -... todo - -### FailedRecordHandler Contract - -... todo - -### Provided Handlers - ## Resources * [Running IcebergSinkConnector locally](https://github.com/wuerike/kafka-iceberg-streaming) diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java index d3319023..4aefcc71 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java @@ -20,7 +20,12 @@ import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; + +import java.util.Collections; import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; @@ -28,6 +33,7 @@ public class DefaultExceptionHandler implements TransformExceptionHandler { private static final String FAILED_RECORD_FACTORY_PROP = "failed_record_factory"; + private static final ConfigDef CONFIG_DEF = new ConfigDef() .define( @@ -45,7 +51,7 @@ public SinkRecord handle(SinkRecord original, Throwable error, String location) } @Override - public void configure(Map props) { + public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); ClassLoader loader = this.getClass().getClassLoader(); this.recordFactory = @@ -58,4 +64,21 @@ public void configure(Map props) { public ConfigDef config() { return CONFIG_DEF; } + + private Map failedRecordProperties(Map originalProps) { + return propertiesWithPrefix(originalProps, FAILED_RECORD_FACTORY_PROP + "."); + } + + private static Map propertiesWithPrefix( + Map properties, String prefix) { + if (properties == null || properties.isEmpty()) { + return Collections.emptyMap(); + } + + Preconditions.checkArgument(prefix != null, "Invalid prefix: null"); + + return properties.entrySet().stream() + .filter(e -> e.getKey().startsWith(prefix)) + .collect(Collectors.toMap(e -> e.getKey().replaceFirst(prefix, ""), Map.Entry::getValue)); + } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index 710f0c53..eb0c6fd0 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -26,6 +26,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Schema; @@ -55,7 +56,7 @@ * *

"transforms": "tab", "transforms.tab.type": *

    - *
  • "io.tabular.iceberg.connect.transform.managed.ManagedTransform"
  • + *
  • "io.tabular.iceberg.connect.transforms.ErrorTransform"
  • *
  • "transforms.tab.value.converter": "org.apache.kafka.connect.storage.StringConverter"
  • *
  • "transforms.tab.value.converter.some_property: "...", "transforms.tab.key.converter":
  • *
  • "org.apache.kafka.connect.storage.StringConverter", "transforms.tab.key.converter.some_property": "..."
  • @@ -64,12 +65,13 @@ *
*

This should not be used with any other SMT. All SMTs should be added to "transforms.tab.smts". * - *

It returns a special Map of String -> Object "original" : Map of String -> Object containing - * the key,value, and header bytes of the original message "transformed" : [null, Struct, Map, etc.] - * of whatever the deserialized record is (after transformation if SMTs are configured) + *

In the success path, the original key/value/header bytes are put on the headers of the transformed record. + * Note that the original Kafka headers are lost due to Kafka Connect; however, the Kafka Connect headers are + * translated and can be used to recover the original Kafka values in order to construct a new Producer Record if required + * (user responsibility). * *

The original payload can be used in the Iceberg Connector if the record cannot be transformed - * to an Iceberg record so that the original kafka message can be stored in Iceberg at that point. + * to an Iceberg record, or some other issue arises, so that the original kafka message can be stored in Iceberg at that point. * *

If any of the key, value, header deserializers or SMTs throw an exception a failed record is * constructed that contains kafka metadata, exception/location information, and the original @@ -297,12 +299,15 @@ record -> { .collect(Collectors.toList()); } + + Map stringProps = propsAsStrings(props); + converterErrorHandler = (TransformExceptionHandler) loadClass(config.getString(CONVERTER_ERROR_HANDLER), loader); - converterErrorHandler.configure(props); + converterErrorHandler.configure(stringProps); smtErrorHandler = (TransformExceptionHandler) loadClass(config.getString(SMT_ERROR_HANDLER), loader); - smtErrorHandler.configure(props); + smtErrorHandler.configure(stringProps); } private Object loadClass(String name, ClassLoader loader) { @@ -381,4 +386,14 @@ private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { return transformed; } + + private Map propsAsStrings(Map props) { + Map newProps = Maps.newHashMap(); + props.forEach((key, value) -> { + if (value instanceof String) { + newProps.put(key, (String) value); + } + }); + return newProps; + } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java index ee5bf881..736f9d44 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java @@ -25,7 +25,7 @@ public interface TransformExceptionHandler { SinkRecord handle(SinkRecord original, Throwable error, String location); - void configure(Map props); + void configure(Map props); ConfigDef config(); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index 77276ea9..d0f306ca 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,11 +83,11 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; - private static final String WRITE_EXCEPTION_HANDLER_PROP = "iceberg.tables.deadletter.handler"; + private static final String WRITE_EXCEPTION_HANDLER_PROP = "iceberg.deadletter.handler"; private static final String FAILED_RECORD_FACTORY_PROP = - "iceberg.tables.deadletter.record_factory"; + "iceberg.deadletter.failed_record_factory"; private static final String FAILED_RECORD_FACTORY_PREFIX = - "iceberg.tables.deadletter.record_factory"; + "iceberg.deadletter.failed_record_factory"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; From e4977c4b35c5d53905a59904c6d79f984aef2bb1 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 5 Jun 2024 11:56:42 -0700 Subject: [PATCH 30/32] dead-letter-table - simplify original bytes location - clean up unused methods, values - simplify write exception interface - move failed record handler initialization to be write exception handler specific - clean up error transform converters - close SMT and converters --- README.md | 43 ++-- kafka-connect-deadletter/build.gradle | 3 - .../connect/deadletter/DeadLetterUtils.java | 23 ++- .../DefaultFailedRecordFactory.java | 53 ++--- .../deadletter/FailedRecordFactory.java | 11 +- .../transforms/DefaultExceptionHandler.java | 29 +-- .../connect/transforms/ErrorTransform.java | 188 ++++++++---------- .../transforms/TransformExceptionHandler.java | 5 +- .../transforms/ErrorTransformTest.java | 48 ++--- .../iceberg/connect/IcebergSinkConfig.java | 19 +- ...DeadLetterTableWriteExceptionHandler.java} | 14 +- .../connect/data/IcebergWriterFactory.java | 3 + .../iceberg/connect/data/RecordRouter.java | 7 +- .../connect/data/WriteExceptionHandler.java | 3 +- .../connect/data/RecordRouterTest.java | 25 ++- 15 files changed, 217 insertions(+), 257 deletions(-) rename kafka-connect/src/main/java/io/tabular/iceberg/connect/data/{DefaultWriteExceptionHandler.java => DeadLetterTableWriteExceptionHandler.java} (84%) diff --git a/README.md b/README.md index a178c231..6e000396 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ The zip archive will be found under `./kafka-connect-runtime/build/distributions | iceberg.hadoop.* | Properties passed through to the Hadoop configuration | | iceberg.kafka.* | Properties passed through to control topic Kafka client initialization | | iceberg.tables.deadletter.handler | See Dead Letter Table Mode | -| iceberg.tables.deadletter.record_factpry | See Dead Letter Table Mode | +| iceberg.tables.deadletter.record_factory | See Dead Letter Table Mode | | iceberg.tables.deadletter.record_factory.* | see Dead Letter Table Mode | If `iceberg.tables.dynamic-enabled` is `false` (the default) then you must specify `iceberg.tables`. If @@ -325,24 +325,24 @@ See above for creating the table } ``` -## Dead Letter Table +## Write Exception Handler Mode -The connector can be configured to write to one or more Dead Letter iceberg tables, with capability beyond +The connector can be configured to use configured exception handlers when errors occur, with capability beyond what is offered from Kafka Connects Dead Letter Queue implementation. This is an optional setting you can ignore if you want failures to cause the connector to fail. Alternatively, the Dead Letter Table mode can be used to ignore all failures, beyond `error.tolerance = all` -| Location of Failure | Kafka Connect DLQ | Dead Letter Table Mode | -|----------------------------------------------------------|-------------------|------------------------| -| Deserialization/Converter | Yes | Yes* | -| SMT | Yes | Yes* | -| Table creation / schema issues | No | Yes | -| Iceberg record conversion | No | Yes | -| Malformed records (e.g. missing table route information) | No | Yes | -| Schema evolution issues | No | Yes | +| Location of Failure | Kafka Connect DLQ | Write Exception Handler Mode | +|----------------------------------------------------------|-------------------|------------------------------| +| Deserialization/Converter | Yes | Yes* | +| SMT | Yes | Yes* | +| Table creation / schema issues | No | Yes | +| Iceberg record conversion | No | Yes | +| Malformed records (e.g. missing table route information) | No | Yes | +| Schema evolution issues | No | Yes | -If the `ErrorTransform` SMT is not used, it may be challenging to put records into the Dead Letter Table other than -recording metadata (Topic, Partition, Offset) while dropping the message. +If the `ErrorTransform` SMT is not used, it is not possible to recover the original message bytes. The only information +available will be recording metadata (Topic, Partition, Offset) while dropping the message. If the `ErrorTransform` SMT is used, failures can include the original bytes of the message in the Iceberg Table that can be extracted/inspected using a downstream query engine. @@ -352,15 +352,15 @@ There are several classes that can be implemented and passed to the config for u * Error Transform SMT Key, Value, and Header deserialization failure handlers (each can be a different class) * SMT transformation failure handlers * Connector WriteException handler to handle issues with records themselves (e.g. missing route columns, invalid schema evolutions, etc.) -* Dead Letter Table schema / Record conversion -Some default implementations are provided. +Some default implementations are provided. The default implementation provides a **DEAD LETTER TABLE** mode that writes the original +message bytes, as well as Kafka metadata, to a configured Iceberg table. -NOTE: **Avro/Schema Registry** should not be used in conjunction with Dead Letter Table using the provided handlers. Avro deserialization +NOTE: **Avro/Schema Registry** should not be used in conjunction with write exception handling using the provided handlers. Avro deserialization failures mix transient and non-transient errors in the same exception. A failure handler for avro deserialization is planned, but not yet implemented. -Users can extend these handlers to suit the needs of their particular deserialization method, catalog, etc. all of whihc can raise +Users can extend these handlers to suit the needs of their particular deserialization method, catalog, etc. all of which can raise different errors. It is advised to carefully develop these and start conservatively: you do not want to send messages to the Dead Letter Table for transient network errors, which may be catalog specific. @@ -394,11 +394,10 @@ The reference implementation of `io.tabular.iceberg.connect.transforms.DefaultEx In order to turn on Dead Letter Table mode in the connector: -| Property | Description | -|---------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------| -| iceberg.deadletter.handler | Sucblass of io.tabular.iceberg.connect.data.WriteExceptionHandler , if this is not-null Dead Letter Table mode is turned on in the connector | -| iceberg.deadletter.failed_record_factory | Scubclass of io.tabular.iceberg.connect.deadletter.FailedRecordFactory for converting records. The connector cannot see the SMT version | -| iceberg.deadletter.failed_record_factory.* | properties to be passed to the failed record factory | +| Property | Description | +|------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------| +| iceberg.write-exception.handler | Sucblass of io.tabular.iceberg.connect.data.WriteExceptionHandler , if this is not-null write exception mode is turned on in the connector | +| iceberg.write-exception.handler.properties.* | properties to be passed to the failed record factory | You do not need to use the Error SMT to turn on dead letter mode; however, the provided `io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory` assumes the Error SMT has been used and will throw exceptions if not. You can implement your own WriteExceptionHandler/FailedRecordFactory to skip messages, transform messages, strip fields from messages and only write the diff --git a/kafka-connect-deadletter/build.gradle b/kafka-connect-deadletter/build.gradle index 93ca7c16..74a6d00f 100644 --- a/kafka-connect-deadletter/build.gradle +++ b/kafka-connect-deadletter/build.gradle @@ -3,9 +3,6 @@ plugins { } dependencies { - implementation libs.iceberg.core - implementation libs.iceberg.common - implementation libs.iceberg.guava compileOnly libs.bundles.kafka.connect } diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java index 394d9738..1377db86 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java @@ -20,8 +20,8 @@ import java.io.PrintWriter; import java.io.StringWriter; +import java.util.ArrayList; import java.util.List; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; @@ -51,19 +51,25 @@ public Throwable getError() { private DeadLetterUtils() {} - public static final String KEY_HEADER = "t_original_key"; - public static final String VALUE_HEADER = "t_original_value"; - - public static final String HEADERS_HEADER = "t_original_headers"; + public static final String ORIGINAL_DATA = "iceberg.error.transform.original"; + public static final String KEY = "KEY"; + public static final String VALUE = "VALUE"; + public static final String HEADERS = "HEADERS"; public static final Schema HEADER_ELEMENT_SCHEMA = SchemaBuilder.struct() .field("key", Schema.STRING_SCHEMA) .field("value", Schema.OPTIONAL_BYTES_SCHEMA) .optional() .build(); - public static final Schema HEADER_SCHEMA = - SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); + SchemaBuilder.array(HEADER_ELEMENT_SCHEMA).optional().build(); + public static final Schema HEADER_STRUCT_SCHEMA = + SchemaBuilder.struct() + .field(KEY, Schema.OPTIONAL_BYTES_SCHEMA) + .field(VALUE, Schema.OPTIONAL_BYTES_SCHEMA) + .field(HEADERS, HEADER_SCHEMA) + .optional() + .build(); public static String stackTrace(Throwable error) { StringWriter sw = new StringWriter(); @@ -82,7 +88,8 @@ public static String stackTrace(Throwable error) { * @return Struct for an Array that can be put into Iceberg */ public static List serializedHeaders(SinkRecord original) { - List headers = Lists.newArrayList(); + @SuppressWarnings("RegexpSingleline") + List headers = new ArrayList<>(); for (Header header : original.headers()) { Struct headerStruct = new Struct(HEADER_ELEMENT_SCHEMA); headerStruct.put("key", header.key()); diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java index cd0c8810..88d0194e 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java @@ -39,12 +39,14 @@ public class DefaultFailedRecordFactory implements FailedRecordFactory { ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "dead letter table name namespace.table") - .define(DEAD_LETTER_ROUTE_FIELD_PROP, - ConfigDef.Type.STRING, - null, - ConfigDef.Importance.MEDIUM, - "route field to inject table name on"); + "dead letter table name namespace.table" + ) + .define(DEAD_LETTER_ROUTE_FIELD_PROP, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + "route field to inject table name on" + ); private static final String HEADERS = "headers"; private Schema schema; @@ -53,14 +55,14 @@ public class DefaultFailedRecordFactory implements FailedRecordFactory { private String deadLetterRouteField; @Override - public Schema schema(String context) { + public Schema schema() { return schema; } @Override - public SinkRecord recordFromSmt(SinkRecord original, Throwable error, String context) { + public SinkRecord recordFromSmt(SinkRecord original, Throwable error) { Struct struct = new Struct(schema); - addCommon(struct, original, error, context); + addCommon(struct, original, error); if (original.key() != null) { struct.put("key_bytes", original.key()); @@ -83,24 +85,28 @@ public SinkRecord recordFromSmt(SinkRecord original, Throwable error, String con } @Override - public SinkRecord recordFromConnector(SinkRecord record, Throwable error, String context) { + public SinkRecord recordFromConnector(SinkRecord record, Throwable error) { Struct struct = new Struct(schema); - addCommon(struct, record, error, context); + addCommon(struct, record, error); Headers headers = record.headers(); - Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); - Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); - Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); - if (keyHeader != null) { - struct.put("key_bytes", keyHeader.value()); + Header originalData = headers.lastWithName(DeadLetterUtils.ORIGINAL_DATA); + Struct originalStruct = (Struct) originalData.value(); + + Object keyBytes = originalStruct.get(DeadLetterUtils.KEY); + Object valueBytes = originalStruct.get(DeadLetterUtils.VALUE); + Object serializedHeader = originalStruct.get(DeadLetterUtils.HEADERS); + + if (keyBytes != null) { + struct.put("key_bytes", keyBytes); } - if (valueHeader != null) { - struct.put("value_bytes", valueHeader.value()); + if (valueBytes != null) { + struct.put("value_bytes", valueBytes); } if (serializedHeader != null) { - struct.put(HEADERS, serializedHeader.value()); + struct.put(HEADERS, serializedHeader); } return record.newRecord( @@ -117,8 +123,6 @@ public void configure(Map props) { } schema = SchemaBuilder.struct() - .name("failed_message") - .parameter("transform_failed", "true") .field("topic", Schema.STRING_SCHEMA) .field("partition", Schema.INT32_SCHEMA) .field("offset", Schema.INT64_SCHEMA) @@ -128,13 +132,12 @@ public void configure(Map props) { .field("key_bytes", Schema.OPTIONAL_BYTES_SCHEMA) .field("value_bytes", Schema.OPTIONAL_BYTES_SCHEMA) .field(HEADERS, DeadLetterUtils.HEADER_SCHEMA) - .field("context", Schema.OPTIONAL_STRING_SCHEMA) .field("target_table", Schema.OPTIONAL_STRING_SCHEMA) .field(deadLetterRouteField, Schema.STRING_SCHEMA) .build(); } - private void addCommon(Struct struct, SinkRecord record, Throwable error, String context) { + private void addCommon(Struct struct, SinkRecord record, Throwable error) { struct.put("topic", record.topic()); struct.put("partition", record.kafkaPartition()); struct.put("offset", record.kafkaOffset()); @@ -144,9 +147,7 @@ private void addCommon(Struct struct, SinkRecord record, Throwable error, String if (!stack.isEmpty()) { struct.put("stack_trace", stack); } - if (context != null) { - struct.put("context", context); - } + struct.put(deadLetterRouteField, deadLetterTableName); } } diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java index db4c4ba0..f1405613 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java +++ b/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java @@ -23,14 +23,13 @@ import org.apache.kafka.connect.sink.SinkRecord; public interface FailedRecordFactory { - Schema schema(String context); + Schema schema(); - // how to take SMT record (which FYI is all ByteArrays) and turn it into some form of FailedRecord - SinkRecord recordFromSmt(SinkRecord original, Throwable error, String context); + // how to take SMT record (which is all ByteArrays) and turn it into some form of FailedRecord + SinkRecord recordFromSmt(SinkRecord original, Throwable error); - // here is where it starts getting awkward - // where in the original are the byte arrays. - SinkRecord recordFromConnector(SinkRecord record, Throwable error, String context); + // how to take a record that fails in the connector and turn it into a FailedRecord + SinkRecord recordFromConnector(SinkRecord record, Throwable error); void configure(Map props); } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java index 4aefcc71..4b89c0f7 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java @@ -21,11 +21,8 @@ import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; -import java.util.Collections; import java.util.Map; -import java.util.stream.Collectors; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; @@ -46,8 +43,8 @@ public class DefaultExceptionHandler implements TransformExceptionHandler { private FailedRecordFactory recordFactory; @Override - public SinkRecord handle(SinkRecord original, Throwable error, String location) { - return recordFactory.recordFromSmt(original, error, location); + public SinkRecord handle(SinkRecord original, Throwable error) { + return recordFactory.recordFromSmt(original, error); } @Override @@ -59,26 +56,4 @@ public void configure(Map props) { DeadLetterUtils.loadClass(config.getString(FAILED_RECORD_FACTORY_PROP), loader); recordFactory.configure(props); } - - @Override - public ConfigDef config() { - return CONFIG_DEF; - } - - private Map failedRecordProperties(Map originalProps) { - return propertiesWithPrefix(originalProps, FAILED_RECORD_FACTORY_PROP + "."); - } - - private static Map propertiesWithPrefix( - Map properties, String prefix) { - if (properties == null || properties.isEmpty()) { - return Collections.emptyMap(); - } - - Preconditions.checkArgument(prefix != null, "Invalid prefix: null"); - - return properties.entrySet().stream() - .filter(e -> e.getKey().startsWith(prefix)) - .collect(Collectors.toMap(e -> e.getKey().replaceFirst(prefix, ""), Map.Entry::getValue)); - } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index eb0c6fd0..0ed4447a 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -23,15 +23,11 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.connect.connector.ConnectRecord; -import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; @@ -41,6 +37,8 @@ import org.apache.kafka.connect.storage.HeaderConverter; import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Wraps key, value, header converters and SMTs in order to catch exceptions. Failed records are @@ -79,6 +77,8 @@ */ public class ErrorTransform implements Transformation { + private static final Logger LOG = LoggerFactory.getLogger(ErrorTransform.class); + public static class TransformInitializationException extends RuntimeException { TransformInitializationException(String errorMessage) { super(errorMessage); @@ -126,18 +126,12 @@ public boolean isFailed() { private static final String CONVERTER_ERROR_HANDLER = "error.converter"; private static final String SMT_ERROR_HANDLER = "error.smt"; private static final String TRANSFORMATIONS = "smts"; - private static final String KEY_FAILURE = "KEY_CONVERTER"; - private static final String VALUE_FAILURE = "VALUE_CONVERTER"; - private static final String HEADER_FAILURE = "HEADER_CONVERTER"; - private static final String SMT_FAILURE = "SMT_FAILURE"; - private static final Schema OPTIONAL_BYTES_SCHEMA = SchemaBuilder.OPTIONAL_BYTES_SCHEMA; - private TransformExceptionHandler converterErrorHandler; private TransformExceptionHandler smtErrorHandler; - private List> smts; - private Function keyConverter; - private Function valueConverter; - private Function headerConverterFn; + private List> smts = Lists.newArrayList(); + private Converter keyConverter; + private Converter valueConverter; + private HeaderConverter headerConverter; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -194,7 +188,7 @@ public SinkRecord apply(SinkRecord record) { break; } } catch (Exception e) { - return smtErrorHandler.handle(record, e, SMT_FAILURE); + return smtErrorHandler.handle(record, e); } } // SMT could filter out messages @@ -211,7 +205,18 @@ public ConfigDef config() { } @Override - public void close() {} + public void close() { + smts.forEach(smt -> { + try { + smt.close(); + } catch (Exception e) { + LOG.error("error closing SMT", e); + } + }); + tryCloseConverter(headerConverter); + tryCloseConverter(valueConverter); + tryCloseConverter(keyConverter); + } /* Kafka Connect filters the properties it passes to the SMT to @@ -223,68 +228,31 @@ public void configure(Map props) { SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); ClassLoader loader = this.getClass().getClassLoader(); - if (Objects.equals( - config.getString(KEY_CONVERTER), - "org.apache.kafka.connect.converters.ByteArrayConverter")) { - keyConverter = record -> new SchemaAndValue(record.keySchema(), record.value()); - } else { - Converter converter = (Converter) loadClass(config.getString(KEY_CONVERTER), loader); - converter.configure(PropsParser.apply(props, KEY_CONVERTER), true); - keyConverter = record -> converter.toConnectData(record.topic(), (byte[]) record.key()); + + keyConverter = (Converter) loadClass(config.getString(KEY_CONVERTER), loader); + try { + keyConverter.configure(PropsParser.apply(props, KEY_CONVERTER), true); + } catch (Exception e){ + throw new TransformInitializationException("Could not configure key converter", e); } + if (config.getString(VALUE_CONVERTER) == null) { throw new TransformInitializationException( - "ManagedTransformWrapper cannot be used without a defined value converter"); - } else { - Converter converter = (Converter) loadClass(config.getString(VALUE_CONVERTER), loader); - converter.configure(PropsParser.apply(props, VALUE_CONVERTER), false); - valueConverter = record -> converter.toConnectData(record.topic(), (byte[]) record.value()); + "ManagedTransformWrapper cannot be used without a defined value converter"); + } + valueConverter = (Converter) loadClass(config.getString(VALUE_CONVERTER), loader); + try { + valueConverter.configure(PropsParser.apply(props, VALUE_CONVERTER), false); + } catch (Exception e) { + throw new TransformInitializationException("Could not configure value converter", e); } - HeaderConverter headerConverter; - - if (Objects.equals( - config.getString(HEADER_CONVERTER), - "org.apache.kafka.connect.converters.ByteArrayConverter")) { - try (HeaderConverter converter = - (HeaderConverter) - loadClass("org.apache.kafka.connect.converters.ByteArrayConverter", loader)) { - converter.configure(PropsParser.apply(props, HEADER_CONVERTER)); - } catch (Exception e) { - throw new TransformInitializationException( - String.format( - "Error loading header converter class %s", config.getString(HEADER_CONVERTER)), - e); - } - headerConverterFn = ConnectRecord::headers; - } else { - try (HeaderConverter converter = - (HeaderConverter) loadClass(config.getString(HEADER_CONVERTER), loader)) { - converter.configure(PropsParser.apply(props, HEADER_CONVERTER)); - headerConverter = converter; - } catch (Exception e) { - throw new TransformInitializationException( - String.format( - "Error loading header converter class %s", config.getString(HEADER_CONVERTER)), - e); - } - - headerConverterFn = - record -> { - Headers newHeaders = new ConnectHeaders(); - Headers recordHeaders = record.headers(); - if (recordHeaders != null) { - String topic = record.topic(); - for (Header recordHeader : recordHeaders) { - SchemaAndValue schemaAndValue = - headerConverter.toConnectHeader( - topic, recordHeader.key(), (byte[]) recordHeader.value()); - newHeaders.add(recordHeader.key(), schemaAndValue); - } - } - return newHeaders; - }; + headerConverter = (HeaderConverter) loadClass(config.getString(HEADER_CONVERTER), loader); + try { + headerConverter.configure(PropsParser.apply(props, HEADER_CONVERTER)); + } catch (Exception e) { + throw new TransformInitializationException("Could not configure header converter", e); } if (config.getString(TRANSFORMATIONS) == null) { @@ -292,21 +260,21 @@ record -> { } else { smts = - Arrays.stream(config.getString(TRANSFORMATIONS).split(",")) - .map(className -> loadClass(className, loader)) - .map(obj -> (Transformation) obj) - .peek(smt -> smt.configure(PropsParser.apply(props, TRANSFORMATIONS))) - .collect(Collectors.toList()); + Arrays.stream(config.getString(TRANSFORMATIONS).split(",")) + .map(className -> loadClass(className, loader)) + .map(obj -> (Transformation) obj) + .peek(smt -> smt.configure(PropsParser.apply(props, TRANSFORMATIONS))) + .collect(Collectors.toList()); } Map stringProps = propsAsStrings(props); converterErrorHandler = - (TransformExceptionHandler) loadClass(config.getString(CONVERTER_ERROR_HANDLER), loader); + (TransformExceptionHandler) loadClass(config.getString(CONVERTER_ERROR_HANDLER), loader); converterErrorHandler.configure(stringProps); smtErrorHandler = - (TransformExceptionHandler) loadClass(config.getString(SMT_ERROR_HANDLER), loader); + (TransformExceptionHandler) loadClass(config.getString(SMT_ERROR_HANDLER), loader); smtErrorHandler.configure(stringProps); } @@ -330,21 +298,23 @@ private DeserializedRecord deserialize(SinkRecord record) { SchemaAndValue valueData; Headers newHeaders; + String topic = record.topic(); + try { - keyData = keyConverter.apply(record); + keyData = keyConverter.toConnectData(topic, (byte[]) record.key()); } catch (Exception e) { - return new DeserializedRecord(converterErrorHandler.handle(record, e, KEY_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e), true); } try { - valueData = valueConverter.apply(record); + valueData = valueConverter.toConnectData(topic, (byte[]) record.value()); } catch (Exception e) { - return new DeserializedRecord(converterErrorHandler.handle(record, e, VALUE_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e), true); } try { - newHeaders = headerConverterFn.apply(record); + newHeaders = deserializeHeaders(record); } catch (Exception e) { - return new DeserializedRecord(converterErrorHandler.handle(record, e, HEADER_FAILURE), true); + return new DeserializedRecord(converterErrorHandler.handle(record, e), true); } return new DeserializedRecord( @@ -360,30 +330,36 @@ private DeserializedRecord deserialize(SinkRecord record) { false); } + + private Headers deserializeHeaders(SinkRecord record) { + Headers newHeaders = new ConnectHeaders(); + Headers recordHeaders = record.headers(); + if (recordHeaders != null) { + String topic = record.topic(); + for (Header recordHeader : recordHeaders) { + SchemaAndValue schemaAndValue = + headerConverter.toConnectHeader( + topic, recordHeader.key(), (byte[]) recordHeader.value()); + newHeaders.add(recordHeader.key(), schemaAndValue); + } + } + return newHeaders; + } + private SinkRecord newRecord(SinkRecord original, SinkRecord transformed) { + Struct struct = new Struct(DeadLetterUtils.HEADER_STRUCT_SCHEMA); if (!original.headers().isEmpty()) { List serializedHeaders = DeadLetterUtils.serializedHeaders(original); - transformed - .headers() - .add( - DeadLetterUtils.HEADERS_HEADER, - new SchemaAndValue(DeadLetterUtils.HEADER_SCHEMA, serializedHeaders)); + struct.put(DeadLetterUtils.HEADERS, serializedHeaders); } if (original.key() != null) { - transformed - .headers() - .add( - DeadLetterUtils.KEY_HEADER, - new SchemaAndValue(OPTIONAL_BYTES_SCHEMA, original.key())); + struct.put(DeadLetterUtils.KEY, original.key()); } if (original.value() != null) { - transformed - .headers() - .add( - DeadLetterUtils.VALUE_HEADER, - new SchemaAndValue(OPTIONAL_BYTES_SCHEMA, original.value())); + struct.put(DeadLetterUtils.VALUE, original.value()); } + transformed.headers().add(DeadLetterUtils.ORIGINAL_DATA, new SchemaAndValue(DeadLetterUtils.HEADER_STRUCT_SCHEMA, struct)); return transformed; } @@ -396,4 +372,16 @@ private Map propsAsStrings(Map props) { }); return newProps; } + + private void tryCloseConverter(Object converter) { + if (converter != null) { + if (converter instanceof AutoCloseable) { + try { + ((AutoCloseable) converter).close(); + } catch (Exception e) { + LOG.error("error closing converter", e); + } + } + } + } } diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java index 736f9d44..fe8ffc30 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/TransformExceptionHandler.java @@ -19,13 +19,10 @@ package io.tabular.iceberg.connect.transforms; import java.util.Map; -import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.sink.SinkRecord; public interface TransformExceptionHandler { - SinkRecord handle(SinkRecord original, Throwable error, String location); + SinkRecord handle(SinkRecord original, Throwable error); void configure(Map props); - - ConfigDef config(); } diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java index 325c79bc..732a90d0 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -41,17 +41,12 @@ import org.junit.jupiter.api.Test; public class ErrorTransformTest { - private static final String TOPIC = "some-topic"; private static final int PARTITION = 3; private static final long OFFSET = 100; private static final long TIMESTAMP = 1000; private static final String KEY_STRING = "key"; private static final String VALUE_STRING = "value"; - private static final String KEY_JSON = "{\"key\": \"blah\"}"; - private static final String VALUE_JSON = "{\"a\": 1, \"b\": \"b\"}"; - private static final String BYTE_ARRAY_CONVERTER = - "org.apache.kafka.connect.converters.ByteArrayConverter"; private static final String JSON_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; private static final String STRING_CONVERTER = "org.apache.kafka.connect.storage.StringConverter"; @@ -123,14 +118,15 @@ public void deserialize() { assertThat(result.value()).isEqualTo(VALUE_STRING); Headers headers = result.headers(); - Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); - Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); - Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); - - assertThat(keyHeader.value()).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); - assertThat(valueHeader.value()).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); - assertThat(serializedHeader.value()).isInstanceOf(List.class); - List resultHeaders = (List) serializedHeader.value(); + Header originalData = headers.lastWithName(DeadLetterUtils.ORIGINAL_DATA); + + assertThat(originalData.value()).isInstanceOf(Struct.class); + Struct originalStruct = (Struct) originalData.value(); + + assertThat(originalStruct.get(DeadLetterUtils.KEY)).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(originalStruct.get(DeadLetterUtils.VALUE)).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(originalStruct.get(DeadLetterUtils.HEADERS)).isInstanceOf(List.class); + List resultHeaders = (List) originalStruct.get(DeadLetterUtils.HEADERS); assertThat(resultHeaders.size()).isEqualTo(1); assertThat(resultHeaders.get(0).get("key")).isEqualTo("h1k"); assertThat(resultHeaders.get(0).get("value")) @@ -172,13 +168,15 @@ public void smt() { assertThat(result.kafkaOffset()).isEqualTo(OFFSET); Headers headers = result.headers(); - Header keyHeader = headers.lastWithName(DeadLetterUtils.KEY_HEADER); - Header valueHeader = headers.lastWithName(DeadLetterUtils.VALUE_HEADER); - Header serializedHeader = headers.lastWithName(DeadLetterUtils.HEADERS_HEADER); + Header originalData = headers.lastWithName(DeadLetterUtils.ORIGINAL_DATA); + + assertThat(originalData.value()).isInstanceOf(Struct.class); + Struct originalStruct = (Struct) originalData.value(); + + assertThat(originalStruct.get(DeadLetterUtils.KEY)).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(originalStruct.get(DeadLetterUtils.VALUE)).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); + assertThat(originalStruct.get(DeadLetterUtils.HEADERS)).isNull(); - assertThat(keyHeader.value()).isEqualTo(KEY_STRING.getBytes(StandardCharsets.UTF_8)); - assertThat(valueHeader.value()).isEqualTo(VALUE_STRING.getBytes(StandardCharsets.UTF_8)); - assertThat(serializedHeader).isNull(); } } @@ -248,8 +246,7 @@ public void keyFailed() { SinkRecord result = smt.apply(createRecord(malformedKey, VALUE_STRING, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); - assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema()); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); @@ -293,8 +290,7 @@ public void valueFailed() { SinkRecord result = smt.apply(createRecord(KEY_STRING, malformedValue, stringAsByteHeaders())); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); - assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema()); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); @@ -345,8 +341,7 @@ public void headerFailed() { SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, headers); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); - assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema()); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); @@ -393,8 +388,7 @@ public void smtFailed() { SinkRecord record = createRecord(KEY_STRING, VALUE_STRING, stringAsByteHeaders()); SinkRecord result = smt.apply(record); assertThat(result.keySchema()).isNull(); - assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema(null)); - assertThat(result.valueSchema().name()).isEqualTo("failed_message"); + assertThat(result.valueSchema()).isEqualTo(failedRecordFactory.schema()); assertThat(result.value()).isInstanceOf(Struct.class); Struct value = (Struct) result.value(); assertThat(value.get("topic")).isEqualTo(TOPIC); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java index d0f306ca..05f813e6 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java @@ -83,11 +83,8 @@ public class IcebergSinkConfig extends AbstractConfig { "iceberg.tables.schema-force-optional"; private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; - private static final String WRITE_EXCEPTION_HANDLER_PROP = "iceberg.deadletter.handler"; - private static final String FAILED_RECORD_FACTORY_PROP = - "iceberg.deadletter.failed_record_factory"; - private static final String FAILED_RECORD_FACTORY_PREFIX = - "iceberg.deadletter.failed_record_factory"; + private static final String WRITE_EXCEPTION_HANDLER_PROP = "iceberg.writer-exception.handler"; + public static final String WRITE_EXCEPTION_HANDLER_PREFIX = "iceberg.write-exception.handler.properties"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; @@ -246,11 +243,11 @@ private static ConfigDef newConfigDef() { Importance.MEDIUM, "If writing to Dead Letter Table, write exception handler class to use"); configDef.define( - FAILED_RECORD_FACTORY_PROP, + WRITE_EXCEPTION_HANDLER_PREFIX, Type.STRING, null, Importance.MEDIUM, - "If writing to Dead Letter Table, failed record factory class to use"); + "If writing to Dead Letter Table, properties to pass during initialization"); return configDef; } @@ -355,10 +352,6 @@ public String getWriteExceptionHandler() { return getString(WRITE_EXCEPTION_HANDLER_PROP); } - public String getFailedRecordHandler() { - return getString(FAILED_RECORD_FACTORY_PROP); - } - public String tablesRouteField() { return getString(TABLES_ROUTE_FIELD_PROP); } @@ -375,8 +368,8 @@ public String tablesDefaultPartitionBy() { return getString(TABLES_DEFAULT_PARTITION_BY); } - public Map failedRecordHandlerProperties() { - return PropertyUtil.propertiesWithPrefix(originalProps, FAILED_RECORD_FACTORY_PREFIX + "."); + public Map writeExceptionHandlerProperties() { + return PropertyUtil.propertiesWithPrefix(originalProps, WRITE_EXCEPTION_HANDLER_PREFIX + "."); } public TableSinkConfig tableConfig(String tableName) { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java similarity index 84% rename from kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java rename to kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java index c78b5535..7cde0c34 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DefaultWriteExceptionHandler.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java @@ -19,18 +19,24 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; +import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; import org.apache.iceberg.exceptions.ValidationException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -public class DefaultWriteExceptionHandler implements WriteExceptionHandler { +import java.util.Map; + +public class DeadLetterTableWriteExceptionHandler implements WriteExceptionHandler { private FailedRecordFactory factory; @Override public void initialize( - SinkTaskContext context, IcebergSinkConfig config, FailedRecordFactory recordFactory) { - this.factory = recordFactory; + SinkTaskContext context, IcebergSinkConfig config) { + Map props = config.writeExceptionHandlerProperties(); + String failedRecordFactoryClass = props.get("failed_record_factory"); + factory = (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, this.getClass().getClassLoader()); + factory.configure(config.writeExceptionHandlerProperties()); } @Override @@ -82,6 +88,6 @@ private SinkRecord handleWriteException(SinkRecord record, WriteException except } private SinkRecord failedRecord(SinkRecord record, WriteException exception) { - return factory.recordFromConnector(record, exception, null); + return factory.recordFromConnector(record, exception); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index 927bb05c..e724718c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -29,9 +29,12 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.util.Tasks; import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class IcebergWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final IcebergSinkConfig config; private final CatalogApi catalogApi; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 0b470819..01c4307b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -20,7 +20,6 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; -import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; import java.util.List; @@ -76,14 +75,10 @@ public static RecordRouter from( } if (config.deadLetterTableEnabled()) { - String failedRecordFactoryClass = config.getFailedRecordHandler(); String handlerClass = config.getWriteExceptionHandler(); - FailedRecordFactory factory = - (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, loader); - factory.configure(config.failedRecordHandlerProperties()); WriteExceptionHandler handler = (WriteExceptionHandler) DeadLetterUtils.loadClass(handlerClass, loader); - handler.initialize(context, config, factory); + handler.initialize(context, config); baseRecordRouter = new RecordRouter.ErrorHandlingRecordRouter(baseRecordRouter, handler); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java index d6cb12f4..591a84fd 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java @@ -19,12 +19,11 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; public interface WriteExceptionHandler { - void initialize(SinkTaskContext context, IcebergSinkConfig config, FailedRecordFactory factory); + void initialize(SinkTaskContext context, IcebergSinkConfig config); /** * This method will be invoked whenever the connector runs into an exception while trying to write diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java index 7811ab7d..21a6abc0 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -38,6 +38,7 @@ import java.nio.charset.StandardCharsets; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; public class RecordRouterTest { @@ -181,16 +182,21 @@ public void errorHandlingRouterGetsConfiguredProperly() { // defaultRecordFactory assumes ErrorTransform has been used to put original bytes on the records in the headers // since this record will fail, it will go through the configured DefaultFailedRecordFactory and expects these values to be present - recordWithoutRoute.headers().add(DeadLetterUtils.VALUE_HEADER, new SchemaAndValue(SchemaBuilder.OPTIONAL_BYTES_SCHEMA, "test".getBytes(StandardCharsets.UTF_8))); + Struct originalRecordStruct = new Struct(DeadLetterUtils.HEADER_STRUCT_SCHEMA); + originalRecordStruct.put(DeadLetterUtils.VALUE,"test".getBytes(StandardCharsets.UTF_8)); + recordWithoutRoute.headers().add(DeadLetterUtils.ORIGINAL_DATA, new SchemaAndValue(DeadLetterUtils.HEADER_STRUCT_SCHEMA, originalRecordStruct)); IcebergSinkConfig config = mock(IcebergSinkConfig.class); when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); when(config.deadLetterTableEnabled()).thenReturn(true); when(config.tablesRouteField()).thenReturn("route_field"); when(config.dynamicTablesEnabled()).thenReturn(true); - when(config.getFailedRecordHandler()).thenReturn("io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"); - when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DefaultWriteExceptionHandler"); - when(config.failedRecordHandlerProperties()).thenReturn(ImmutableMap.of("table_name", "dlt.table", "route_field", "route_field")); + + Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field"); + when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); + + + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DeadLetterTableWriteExceptionHandler"); RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); assertThat(router).isInstanceOf(RecordRouter.ErrorHandlingRecordRouter.class); @@ -215,21 +221,22 @@ public void errorHandlingRouterDoesNotInfiniteLoop() { // defaultRecordFactory assumes ErrorTransform has been used to put original bytes on the records in the headers // since this record will fail, it will go through the configured DefaultFailedRecordFactory and expects these values to be present - recordWithoutRoute.headers().add(DeadLetterUtils.VALUE_HEADER, new SchemaAndValue(SchemaBuilder.OPTIONAL_BYTES_SCHEMA, "test".getBytes(StandardCharsets.UTF_8))); + Struct originalRecordStruct = new Struct(DeadLetterUtils.HEADER_STRUCT_SCHEMA); + originalRecordStruct.put(DeadLetterUtils.VALUE,"test".getBytes(StandardCharsets.UTF_8)); + recordWithoutRoute.headers().add(DeadLetterUtils.ORIGINAL_DATA, new SchemaAndValue(DeadLetterUtils.HEADER_STRUCT_SCHEMA, originalRecordStruct)); IcebergSinkConfig config = mock(IcebergSinkConfig.class); when(config.tables()).thenReturn(Lists.newArrayList("tbl1", "tbl2")); when(config.deadLetterTableEnabled()).thenReturn(true); when(config.tablesRouteField()).thenReturn("route_field"); when(config.dynamicTablesEnabled()).thenReturn(true); - when(config.getFailedRecordHandler()).thenReturn("io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory"); - when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DefaultWriteExceptionHandler"); + Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field_bad"); + when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DeadLetterTableWriteExceptionHandler"); // the underlying router is looking for `route_field` but the failed record handler is configured to have // the route field on `route_field_bad` // this should cause the ErrorHandler to throw an exception // since this is a configuration issue, it should kill the connector w/ unhandled exception - when(config.failedRecordHandlerProperties()).thenReturn(ImmutableMap.of("table_name", "dlt.table", "route_field", "route_field_bad")); - RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); assertThat(router).isInstanceOf(RecordRouter.ErrorHandlingRecordRouter.class); From 4036557e59fb19558e4644ef397f9f15068a434e Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 5 Jun 2024 12:12:20 -0700 Subject: [PATCH 31/32] rename module to not include deadletter --- .../build.gradle | 0 .../tabular/iceberg/connect/exception}/DeadLetterUtils.java | 2 +- .../connect/exception}/DefaultFailedRecordFactory.java | 2 +- .../iceberg/connect/exception}/FailedRecordFactory.java | 2 +- kafka-connect-transforms/build.gradle | 2 +- .../iceberg/connect/transforms/DefaultExceptionHandler.java | 6 +++--- .../tabular/iceberg/connect/transforms/ErrorTransform.java | 2 +- .../iceberg/connect/transforms/ErrorTransformTest.java | 6 +++--- kafka-connect/build.gradle | 2 +- .../connect/data/DeadLetterTableWriteExceptionHandler.java | 4 ++-- .../java/io/tabular/iceberg/connect/data/RecordRouter.java | 2 +- .../io/tabular/iceberg/connect/data/RecordRouterTest.java | 6 +++--- settings.gradle | 4 ++-- 13 files changed, 20 insertions(+), 20 deletions(-) rename {kafka-connect-deadletter => kafka-connect-exception}/build.gradle (100%) rename {kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/DeadLetterUtils.java (98%) rename {kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/DefaultFailedRecordFactory.java (99%) rename {kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/FailedRecordFactory.java (96%) diff --git a/kafka-connect-deadletter/build.gradle b/kafka-connect-exception/build.gradle similarity index 100% rename from kafka-connect-deadletter/build.gradle rename to kafka-connect-exception/build.gradle diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterUtils.java similarity index 98% rename from kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterUtils.java index 1377db86..447d870a 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DeadLetterUtils.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.deadletter; +package io.tabular.iceberg.connect.exception; import java.io.PrintWriter; import java.io.StringWriter; diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DefaultFailedRecordFactory.java similarity index 99% rename from kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DefaultFailedRecordFactory.java index 88d0194e..47426880 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/DefaultFailedRecordFactory.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DefaultFailedRecordFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.deadletter; +package io.tabular.iceberg.connect.exception; import java.util.Map; import org.apache.kafka.common.config.ConfigDef; diff --git a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/FailedRecordFactory.java similarity index 96% rename from kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/FailedRecordFactory.java index f1405613..bcba6ecd 100644 --- a/kafka-connect-deadletter/src/main/java/io/tabular/iceberg/connect/deadletter/FailedRecordFactory.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/FailedRecordFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.deadletter; +package io.tabular.iceberg.connect.exception; import java.util.Map; import org.apache.kafka.connect.data.Schema; diff --git a/kafka-connect-transforms/build.gradle b/kafka-connect-transforms/build.gradle index 74bcf40b..3997d735 100644 --- a/kafka-connect-transforms/build.gradle +++ b/kafka-connect-transforms/build.gradle @@ -1,5 +1,5 @@ dependencies { - implementation project(":iceberg-kafka-connect-deadletter") + implementation project(":iceberg-kafka-connect-exception") implementation libs.iceberg.guava implementation libs.bson implementation libs.slf4j diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java index 4b89c0f7..c5057702 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/DefaultExceptionHandler.java @@ -18,8 +18,8 @@ */ package io.tabular.iceberg.connect.transforms; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; -import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.FailedRecordFactory; import java.util.Map; @@ -36,7 +36,7 @@ public class DefaultExceptionHandler implements TransformExceptionHandler { .define( FAILED_RECORD_FACTORY_PROP, ConfigDef.Type.STRING, - "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory", + "io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory", ConfigDef.Importance.MEDIUM, "class name for failed record conversion"); diff --git a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java index 0ed4447a..7f345d91 100644 --- a/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java +++ b/kafka-connect-transforms/src/main/java/io/tabular/iceberg/connect/transforms/ErrorTransform.java @@ -18,7 +18,7 @@ */ package io.tabular.iceberg.connect.transforms; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; import java.util.Arrays; import java.util.List; import java.util.Map; diff --git a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java index 732a90d0..fc077b40 100644 --- a/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java +++ b/kafka-connect-transforms/src/test/java/io/tabular/iceberg/connect/transforms/ErrorTransformTest.java @@ -21,8 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; -import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.FailedRecordFactory; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -86,7 +86,7 @@ private FailedRecordFactory getFailedRecordFactory() { FailedRecordFactory factory = (FailedRecordFactory) DeadLetterUtils.loadClass( - "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory", + "io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory", this.getClass().getClassLoader()); factory.configure(ImmutableMap.of("table_name", DEAD_LETTER_TABLE_NAME, DEAD_LETTER_ROUTE_FIELD_PROP, DEAD_LETTER_ROUTE_FIELD)); return factory; diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 759e1eed..f7a2f73c 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -1,6 +1,6 @@ dependencies { implementation project(":iceberg-kafka-connect-events") - implementation project(":iceberg-kafka-connect-deadletter") + implementation project(":iceberg-kafka-connect-exception") implementation libs.bundles.iceberg implementation libs.bundles.jackson implementation libs.avro diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java index 7cde0c34..0c1a34da 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java @@ -19,8 +19,8 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; -import io.tabular.iceberg.connect.deadletter.FailedRecordFactory; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.FailedRecordFactory; import org.apache.iceberg.exceptions.ValidationException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 01c4307b..0315a00e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -19,7 +19,7 @@ package io.tabular.iceberg.connect.data; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; import java.util.List; diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java index 21a6abc0..a77dadbb 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -21,7 +21,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.deadletter.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.DeadLetterUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; @@ -192,7 +192,7 @@ public void errorHandlingRouterGetsConfiguredProperly() { when(config.tablesRouteField()).thenReturn("route_field"); when(config.dynamicTablesEnabled()).thenReturn(true); - Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field"); + Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field"); when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); @@ -230,7 +230,7 @@ public void errorHandlingRouterDoesNotInfiniteLoop() { when(config.deadLetterTableEnabled()).thenReturn(true); when(config.tablesRouteField()).thenReturn("route_field"); when(config.dynamicTablesEnabled()).thenReturn(true); - Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field_bad"); + Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field_bad"); when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DeadLetterTableWriteExceptionHandler"); // the underlying router is looking for `route_field` but the failed record handler is configured to have diff --git a/settings.gradle b/settings.gradle index 9820da4e..3e701557 100644 --- a/settings.gradle +++ b/settings.gradle @@ -10,5 +10,5 @@ project(":iceberg-kafka-connect-transforms").projectDir = file("kafka-connect-tr include "iceberg-kafka-connect-runtime" project(":iceberg-kafka-connect-runtime").projectDir = file("kafka-connect-runtime") -include "iceberg-kafka-connect-deadletter" -project(":iceberg-kafka-connect-deadletter").projectDir = file("kafka-connect-deadletter") +include "iceberg-kafka-connect-exception" +project(":iceberg-kafka-connect-exception").projectDir = file("kafka-connect-exception") From 0ff797297c87066d74fbdad88c90eb2f43fc9a03 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Wed, 5 Jun 2024 14:00:21 -0700 Subject: [PATCH 32/32] moved writeExceptions into exception module --- README.md | 10 ++++----- kafka-connect-exception/build.gradle | 1 + .../DeadLetterTableWriteExceptionHandler.java | 10 +++------ .../connect/exception}/WriteException.java | 22 +++++++++---------- .../exception}/WriteExceptionHandler.java | 7 +++--- .../iceberg/connect/data/BaseCatalogApi.java | 2 ++ .../iceberg/connect/data/IcebergWriter.java | 2 ++ .../connect/data/IcebergWriterFactory.java | 2 ++ .../iceberg/connect/data/RecordConverter.java | 2 ++ .../iceberg/connect/data/RecordRouter.java | 4 +++- .../iceberg/connect/data/SchemaUtils.java | 2 ++ .../connect/data/BaseCatalogApiTest.java | 2 ++ .../connect/data/RecordRouterTest.java | 5 +++-- 13 files changed, 42 insertions(+), 29 deletions(-) rename {kafka-connect/src/main/java/io/tabular/iceberg/connect/data => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/DeadLetterTableWriteExceptionHandler.java (89%) rename {kafka-connect/src/main/java/io/tabular/iceberg/connect/data => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/WriteException.java (81%) rename {kafka-connect/src/main/java/io/tabular/iceberg/connect/data => kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception}/WriteExceptionHandler.java (89%) diff --git a/README.md b/README.md index 6e000396..0dd0c14a 100644 --- a/README.md +++ b/README.md @@ -399,9 +399,9 @@ In order to turn on Dead Letter Table mode in the connector: | iceberg.write-exception.handler | Sucblass of io.tabular.iceberg.connect.data.WriteExceptionHandler , if this is not-null write exception mode is turned on in the connector | | iceberg.write-exception.handler.properties.* | properties to be passed to the failed record factory | -You do not need to use the Error SMT to turn on dead letter mode; however, the provided `io.tabular.iceberg.connect.deadletter.DefaultFailedRecordFactory` assumes the Error SMT has been used -and will throw exceptions if not. You can implement your own WriteExceptionHandler/FailedRecordFactory to skip messages, transform messages, strip fields from messages and only write the -Kafka metadata etc. +You do not need to use the Error SMT to turn on dead letter mode; however, the provided `io.tabular.iceberg.connect.exception.DeadLetterTableWriteExceptionHandler` initializes a +`io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory` that assumes the Error SMT has been used and will throw exceptions if not. This handler will write to a configured Iceberg table. +You can implement your own WriteExceptionHandler/FailedRecordFactory to skip messages, transform messages, strip fields from messages and only write the Kafka metadata, etc. ### Routing @@ -416,14 +416,14 @@ FailedRecordHandler that can be used to dispatch to one or more Dead Letter tabl | populated | false | null | ConfigRouter | | populated | false | populated | DynamicRecordRouter then ConfigRouter | -Regardless of the above, if a WriteExceptionHandler `io.tabular.iceberg.connect.data.WriteExceptionHandler` is not null, Dead Letter Table mode +Regardless of the above, if a WriteExceptionHandler `io.tabular.iceberg.connect.data.WriteExceptionHandler` is not null, Write Exception mode will wrap one of the underlying record routing modes. All exceptions are passed to the WriteExceptionHandler where they can be ignored (message dropped), converted into a record and dispatched to the Dead Letter Table(s), or rethrown to fail the connector. ### Partitioning -The following properties still apply to the Dead Letter Table. The partition-by field can be used to customize the +The following properties still apply if the Write Exception handler writes to a Dead Letter table. The partition-by field can be used to customize the partitioning of the Dead Letter table(s). | Property | Description | diff --git a/kafka-connect-exception/build.gradle b/kafka-connect-exception/build.gradle index 74a6d00f..bb9dd66c 100644 --- a/kafka-connect-exception/build.gradle +++ b/kafka-connect-exception/build.gradle @@ -3,6 +3,7 @@ plugins { } dependencies { + implementation libs.iceberg.core compileOnly libs.bundles.kafka.connect } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterTableWriteExceptionHandler.java similarity index 89% rename from kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterTableWriteExceptionHandler.java index 0c1a34da..9a33a133 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/DeadLetterTableWriteExceptionHandler.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/DeadLetterTableWriteExceptionHandler.java @@ -16,11 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.data; +package io.tabular.iceberg.connect.exception; -import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.exception.DeadLetterUtils; -import io.tabular.iceberg.connect.exception.FailedRecordFactory; import org.apache.iceberg.exceptions.ValidationException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -32,11 +29,10 @@ public class DeadLetterTableWriteExceptionHandler implements WriteExceptionHandl @Override public void initialize( - SinkTaskContext context, IcebergSinkConfig config) { - Map props = config.writeExceptionHandlerProperties(); + SinkTaskContext context, Map props) { String failedRecordFactoryClass = props.get("failed_record_factory"); factory = (FailedRecordFactory) DeadLetterUtils.loadClass(failedRecordFactoryClass, this.getClass().getClassLoader()); - factory.configure(config.writeExceptionHandlerProperties()); + factory.configure(props); } @Override diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteException.java similarity index 81% rename from kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteException.java index b1b2123b..4c9737c0 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteException.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteException.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.data; +package io.tabular.iceberg.connect.exception; import org.apache.iceberg.catalog.TableIdentifier; @@ -24,22 +24,22 @@ public class WriteException extends RuntimeException { private final String tableIdentifier; - WriteException(Throwable cause) { + public WriteException(Throwable cause) { super(cause); tableIdentifier = null; } - WriteException(String msg) { + public WriteException(String msg) { super(msg); tableIdentifier = null; } - WriteException(TableIdentifier tableId, Throwable cause) { + public WriteException(TableIdentifier tableId, Throwable cause) { super(cause); this.tableIdentifier = tableId.toString(); } - WriteException(String tableId, Throwable cause) { + public WriteException(String tableId, Throwable cause) { super(cause); this.tableIdentifier = tableId; } @@ -76,36 +76,36 @@ public LoadTableException(TableIdentifier identifier, Throwable cause) { public static class RecordConversionException extends WriteException { - RecordConversionException(Throwable cause) { + public RecordConversionException(Throwable cause) { super(cause); } } public static class RouteException extends WriteException { - RouteException(Throwable cause) { + public RouteException(Throwable cause) { super(cause); } - RouteException(String msg) { + public RouteException(String msg) { super(msg); } } public static class RouteRegexException extends WriteException { - RouteRegexException(Throwable cause) { + public RouteRegexException(Throwable cause) { super(cause); } } public static class SchemaEvolutionException extends WriteException { - SchemaEvolutionException(String name, Throwable cause) { + public SchemaEvolutionException(String name, Throwable cause) { super(name, cause); } } public static class TableIdentifierException extends WriteException { - TableIdentifierException(String name, Throwable cause) { + public TableIdentifierException(String name, Throwable cause) { super(name, cause); } } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteExceptionHandler.java similarity index 89% rename from kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java rename to kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteExceptionHandler.java index 591a84fd..0ecb7860 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/WriteExceptionHandler.java +++ b/kafka-connect-exception/src/main/java/io/tabular/iceberg/connect/exception/WriteExceptionHandler.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.data; +package io.tabular.iceberg.connect.exception; -import io.tabular.iceberg.connect.IcebergSinkConfig; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; +import java.util.Map; + public interface WriteExceptionHandler { - void initialize(SinkTaskContext context, IcebergSinkConfig config); + void initialize(SinkTaskContext context, Map config); /** * This method will be invoked whenever the connector runs into an exception while trying to write diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java index 51cf9527..42e6b983 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/BaseCatalogApi.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Map; import java.util.function.BiFunction; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java index 92b261c4..f3696158 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriter.java @@ -24,6 +24,8 @@ import java.io.UncheckedIOException; import java.util.Arrays; import java.util.List; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java index e724718c..f064726b 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/IcebergWriterFactory.java @@ -20,6 +20,8 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import java.util.concurrent.atomic.AtomicReference; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java index 5809456f..327f9dbf 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordConverter.java @@ -43,6 +43,8 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java index 0315a00e..d61a106c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/RecordRouter.java @@ -23,6 +23,8 @@ import java.util.List; +import io.tabular.iceberg.connect.exception.WriteException; +import io.tabular.iceberg.connect.exception.WriteExceptionHandler; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.kafka.connect.sink.SinkRecord; @@ -78,7 +80,7 @@ public static RecordRouter from( String handlerClass = config.getWriteExceptionHandler(); WriteExceptionHandler handler = (WriteExceptionHandler) DeadLetterUtils.loadClass(handlerClass, loader); - handler.initialize(context, config); + handler.initialize(context, config.writeExceptionHandlerProperties()); baseRecordRouter = new RecordRouter.ErrorHandlingRecordRouter(baseRecordRouter, handler); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java index 0e97de1f..964f789d 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/SchemaUtils.java @@ -35,6 +35,8 @@ import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.UpdateSchema; diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java index cc9522b4..d4ec668b 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/BaseCatalogApiTest.java @@ -27,6 +27,8 @@ import io.tabular.iceberg.connect.TableSinkConfig; import java.util.function.BiFunction; import java.util.regex.Pattern; + +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ValidationException; diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java index a77dadbb..9d2ea781 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/data/RecordRouterTest.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.when; import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.exception.DeadLetterUtils; +import io.tabular.iceberg.connect.exception.WriteException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; @@ -196,7 +197,7 @@ public void errorHandlingRouterGetsConfiguredProperly() { when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); - when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DeadLetterTableWriteExceptionHandler"); + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.exception.DeadLetterTableWriteExceptionHandler"); RecordRouter router = RecordRouter.from(manager, config, this.getClass().getClassLoader(), context); assertThat(router).isInstanceOf(RecordRouter.ErrorHandlingRecordRouter.class); @@ -232,7 +233,7 @@ public void errorHandlingRouterDoesNotInfiniteLoop() { when(config.dynamicTablesEnabled()).thenReturn(true); Map deadLetterProperties = ImmutableMap.of("failed_record_factory", "io.tabular.iceberg.connect.exception.DefaultFailedRecordFactory","table_name", "dlt.table", "route_field", "route_field_bad"); when(config.writeExceptionHandlerProperties()).thenReturn(deadLetterProperties); - when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.data.DeadLetterTableWriteExceptionHandler"); + when(config.getWriteExceptionHandler()).thenReturn("io.tabular.iceberg.connect.exception.DeadLetterTableWriteExceptionHandler"); // the underlying router is looking for `route_field` but the failed record handler is configured to have // the route field on `route_field_bad` // this should cause the ErrorHandler to throw an exception