Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Dead letter table #233

Open
wants to merge 36 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 32 commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
03c3e36
dead-letter-table
tabmatfournier Apr 4, 2024
e87d820
next steps
tabmatfournier Apr 4, 2024
b062c51
moved transform in
tabmatfournier Apr 5, 2024
9b7f6b0
more refactoring
tabmatfournier Apr 8, 2024
c1b4de2
even more refactoring
tabmatfournier Apr 8, 2024
4e706e1
cruft
tabmatfournier Apr 8, 2024
f9136d8
even more cruft
tabmatfournier Apr 8, 2024
d3905a5
tests
tabmatfournier Apr 8, 2024
18c79ba
table create exceptions
tabmatfournier Apr 9, 2024
77aad4b
introduces identifier column, more error handle wrapping
tabmatfournier Apr 9, 2024
edc75a5
make converter/smt error converters configurable/user extensible
tabmatfournier Apr 9, 2024
8ee6840
introduce catalogApi to make IcebergWriterFactory testing easier
tabmatfournier Apr 9, 2024
de479d3
catalogApi and test stubs
tabmatfournier Apr 9, 2024
b78a68d
finished tests
tabmatfournier Apr 10, 2024
ac5d6a5
negate
tabmatfournier Apr 10, 2024
50b300b
dead-letter-table
tabmatfournier Apr 26, 2024
01f8cbb
put null record dropping back into iceberg writer
tabmatfournier Apr 26, 2024
d89f15a
fix dead letter utils private constructor
tabmatfournier Apr 26, 2024
c5c2186
fix cruft in readme
tabmatfournier Apr 26, 2024
831f205
Merge branch 'main' into dead-letter-table
tabmatfournier Apr 29, 2024
451e20f
Merge branch 'main' into dead-letter-table
tabmatfournier Apr 30, 2024
41c4372
post-merge fixes
tabmatfournier Apr 30, 2024
13220e9
more comments removing cruft
tabmatfournier Apr 30, 2024
797b861
regexrecordrouter
tabmatfournier Apr 30, 2024
7bd7d5d
start of fallback mode
tabmatfournier Apr 30, 2024
bff7233
third mode
tabmatfournier May 13, 2024
25208da
another test case
tabmatfournier May 13, 2024
205c2d7
better regex detection to avoid an extra config
tabmatfournier May 13, 2024
4aeedcd
cruft cleanup and starting docs
tabmatfournier May 13, 2024
0cf18d1
fix error transform tests
tabmatfournier May 14, 2024
05ea87f
more docs
tabmatfournier May 14, 2024
457a2f3
Merge branch 'main' into dead-letter-table
tabmatfournier May 21, 2024
1518cc7
Merge branch 'main' into dead-letter-table
tabmatfournier Jun 5, 2024
e4977c4
dead-letter-table
tabmatfournier Jun 5, 2024
4036557
rename module to not include deadletter
tabmatfournier Jun 5, 2024
0ff7972
moved writeExceptions into exception module
tabmatfournier Jun 5, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
111 changes: 111 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -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 |
tabmatfournier marked this conversation as resolved.
Show resolved Hide resolved
| 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
Expand Down Expand Up @@ -322,6 +325,114 @@ See above for creating the table
}
```

## Dead Letter Table

Choose a reason for hiding this comment

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

Example config with Dead Letter will be very useful

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Docs can always be improved. I'll try to take another stab at this.

This is a big feature with a somewhat clunky coinfig API due to config visibility rules in Kafka Connect, so more docs/examples certainly help.

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. 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 |

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.

There are several classes that can be implemented and passed to the config for user customization:

* 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.

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

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.\<table name\>.route-regex set |
| 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
partitioning of the Dead Letter table(s).

| Property | Description |
|--------------------------------------------|------------------------------------------------------------------------------------------------|
| iceberg.table.\<table name\>.commit-branch | Table-specific branch for commits, use `iceberg.tables.default-commit-branch` if not specified |
| iceberg.table.\<table name\>.id-columns | Comma-separated list of columns that identify a row in the table (primary key) |
| iceberg.table.\<table name\>.partition-by | Comma-separated list of partition fields to use when creating the table |

## Resources

* [Running IcebergSinkConnector locally](https://github.com/wuerike/kafka-iceberg-streaming)
1 change: 1 addition & 0 deletions gradle/libs.versions.toml
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,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" }
Expand Down
18 changes: 18 additions & 0 deletions kafka-connect-deadletter/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
plugins {
id "java-test-fixtures"
}

dependencies {
implementation libs.iceberg.core
implementation libs.iceberg.common
implementation libs.iceberg.guava
Copy link
Contributor

Choose a reason for hiding this comment

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

You don't need any of the actual iceberg functionality in this module?

Suggested change
implementation libs.iceberg.core
implementation libs.iceberg.common
implementation libs.iceberg.guava

The only thing you do need is this import :D

import org.apache.iceberg.relocated.com.google.common.collect.Lists;

List<Struct> headers = Lists.newArrayList();

Which IMO you can just replace with this safely.

import java.util.ArrayList;

@SuppressWarnings("RegexpSingleline")
List<Struct> headers = new ArrayList<>();

compileOnly libs.bundles.kafka.connect
}

publishing {
publications {
mavenJava(MavenPublication) {
from components.java
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
/*
* 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 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 {

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() {}

public static final String KEY_HEADER = "t_original_key";
public static final String VALUE_HEADER = "t_original_value";
Copy link
Contributor

Choose a reason for hiding this comment

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

I believe you're de-risking the chance of a collision with an existing header by prefixing a t_
More out of curiosity, what does the t_ stand for?
And wondering if we can do a little better.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, we could de-risk collisions more by just adding a single header t_original_record which is a Struct with this kind of structure (psuedo-code) instead of adding 3 separate headers:

Struct {
   OPTIONAL_BYTES_SCHEMA key, 
   OPTIONAL_BYTES_SCHEMA value, 
   OPTIONAL_ARRAY_HEADER_SCHEMA headers, 
}

nit: I would also name the header something specific to iceberg-kafka-connect IDK something along the lines of kafka.connect.iceberg.error.transform.original.record or something (obviously this is too long but you get the idea).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You are correct in derisking collisions. I chose t for tabular.

Copy link
Contributor

Choose a reason for hiding this comment

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

I chose t for tabular.

🤦 should have figured that one out ....


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();
tabmatfournier marked this conversation as resolved.
Show resolved Hide resolved

public static String stackTrace(Throwable error) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
error.printStackTrace(pw);
return sw.toString();
}

/**
* 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<Struct> serializedHeaders(SinkRecord original) {
List<Struct> 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;
}

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;
}
}
Loading
Loading