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

Kafka Connect: Add SMTs for Debezium and AWS DMS #11936

Open
wants to merge 15 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 14 commits
Commits
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
1 change: 1 addition & 0 deletions .github/workflows/kafka-connect-ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ jobs:
- run: |
./gradlew -DsparkVersions= -DhiveVersions= -DflinkVersions= -DkafkaVersions=3 \
:iceberg-kafka-connect:iceberg-kafka-connect-events:check \
:iceberg-kafka-connect:iceberg-kafka-connect-transforms:check \
:iceberg-kafka-connect:iceberg-kafka-connect:check \
:iceberg-kafka-connect:iceberg-kafka-connect-runtime:check \
-Pquick=true -x javadoc
Expand Down
3 changes: 3 additions & 0 deletions gradle/libs.versions.toml
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ awaitility = "4.2.2"
awssdk-bom = "2.29.43"
azuresdk-bom = "1.2.30"
awssdk-s3accessgrants = "2.3.0"
bson-ver = "4.11.0"
caffeine = "2.9.3"
calcite = "1.10.0"
datasketches = "6.1.1"
Expand Down Expand Up @@ -100,6 +101,7 @@ avro-avro = { module = "org.apache.avro:avro", version.ref = "avro" }
awssdk-bom = { module = "software.amazon.awssdk:bom", version.ref = "awssdk-bom" }
awssdk-s3accessgrants = { module = "software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin", version.ref = "awssdk-s3accessgrants" }
azuresdk-bom = { module = "com.azure:azure-sdk-bom", version.ref = "azuresdk-bom" }
bson = { module = "org.mongodb:bson", version.ref = "bson-ver"}
Copy link
Contributor

Choose a reason for hiding this comment

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

We need to verify this addition is captured in LICENSE/NOTICE files.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added section for mongodb bson libs to LICENSE

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we can remove it until we add the transform library to the runtime. When we do that, then the new license plugin can add what is needed.

caffeine = { module = "com.github.ben-manes.caffeine:caffeine", version.ref = "caffeine" }
calcite-core = { module = "org.apache.calcite:calcite-core", version.ref = "calcite" }
calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "calcite" }
Expand Down Expand Up @@ -159,6 +161,7 @@ jaxb-runtime = { module = "org.glassfish.jaxb:jaxb-runtime", version.ref = "jaxb
kafka-clients = { module = "org.apache.kafka:kafka-clients", version.ref = "kafka" }
kafka-connect-api = { module = "org.apache.kafka:connect-api", version.ref = "kafka" }
kafka-connect-json = { module = "org.apache.kafka:connect-json", version.ref = "kafka" }
kafka-connect-transforms = { module = "org.apache.kafka:connect-transforms", version.ref = "kafka" }
Copy link
Contributor

Choose a reason for hiding this comment

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

We need to verify this addition is captured in LICENSE/NOTICE files (if separate from the other kafka dependencies)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added section for kafka libs to LICENSE

Copy link
Contributor

@bryanck bryanck Jan 17, 2025

Choose a reason for hiding this comment

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

I think we can skip that too for now, until we add the transforms to the runtime.

Copy link
Contributor Author

@ismailsimsek ismailsimsek Jan 23, 2025

Choose a reason for hiding this comment

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

@bryanck do you mean remove all grade setting/changes from this PR and add it with separate PR?

Copy link
Contributor

Choose a reason for hiding this comment

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

I meant skip any of the runtime license changes, until we add the transforms lib to the runtime

Copy link
Contributor Author

Choose a reason for hiding this comment

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

in that case i have reverted all the changes done to LICENSE/NOTICE files.

microprofile-openapi-api = { module = "org.eclipse.microprofile.openapi:microprofile-openapi-api", version.ref = "microprofile-openapi-api" }
nessie-client = { module = "org.projectnessie.nessie:nessie-client", version.ref = "nessie" }
netty-buffer = { module = "io.netty:netty-buffer", version.ref = "netty-buffer" }
Expand Down
17 changes: 17 additions & 0 deletions kafka-connect/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') {

dependencies {
implementation project(':iceberg-kafka-connect:iceberg-kafka-connect')
implementation project(':iceberg-kafka-connect:iceberg-kafka-connect-transforms')
implementation(libs.hadoop3.common) {
exclude group: 'log4j'
exclude group: 'org.slf4j'
Expand Down Expand Up @@ -242,3 +243,19 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') {

assemble.dependsOn distZip, hiveDistZip
}

project(':iceberg-kafka-connect:iceberg-kafka-connect-transforms') {
ismailsimsek marked this conversation as resolved.
Show resolved Hide resolved
dependencies {
implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
implementation libs.bson
implementation libs.slf4j.api
compileOnly libs.kafka.clients
compileOnly libs.kafka.connect.api
compileOnly libs.kafka.connect.json
compileOnly libs.kafka.connect.transforms
}

test {
useJUnitPlatform()
}
}
185 changes: 185 additions & 0 deletions kafka-connect/kafka-connect-transforms/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,185 @@
<!--
- 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.
-->

# SMTs for the Apache Iceberg Sink Connector
Copy link
Contributor

Choose a reason for hiding this comment

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

@bryanck I feel like this should get moved to the docs pages, right?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes that makes sense to me, we should remove this README and move the content to the docs

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added it to "Kafka Connect" page


This project contains some SMTs that could be useful when transforming Kafka data for use by
the Iceberg sink connector.

# CopyValue
_(Experimental)_

The `CopyValue` SMT copies a value from one field to a new field.

## Configuration

| Property | Description |
|------------------|-------------------|
| source.field | Source field name |
| target.field | Target field name |

## Example

```
"transforms": "copyId",
"transforms.copyId.type": "org.apache.iceberg.connect.transforms.CopyValue",
"transforms.copyId.source.field": "id",
"transforms.copyId.target.field": "id_copy",
```

# DmsTransform
_(Experimental)_

The `DmsTransform` SMT transforms an AWS DMS formatted message for use by the sink's CDC feature.
It will promote the `data` element fields to top level and add the following metadata fields:
`_cdc.op`, `_cdc.ts`, and `_cdc.source`.

## Configuration

The SMT currently has no configuration.

# DebeziumTransform
_(Experimental)_

The `DebeziumTransform` SMT transforms a Debezium formatted message for use by the sink's CDC feature.
It will promote the `before` or `after` element fields to top level and add the following metadata fields:
`_cdc.op`, `_cdc.ts`, `_cdc.offset`, `_cdc.source`, `_cdc.target`, and `_cdc.key`.

## Configuration

| Property | Description |
|---------------------|-----------------------------------------------------------------------------------|
| cdc.target.pattern | Pattern to use for setting the CDC target field value, default is `{db}.{table}` |

# JsonToMapTransform
_(Experimental)_

The `JsonToMapTransform` SMT parses Strings as Json object payloads to infer schemas. The iceberg-kafka-connect
connector for schema-less data (e.g. the Map produced by the Kafka supplied JsonConverter) is to convert Maps into Iceberg
Structs. This is fine when the JSON is well-structured, but when you have JSON objects with dynamically
changing keys, it will lead to an explosion of columns in the Iceberg table due to schema evolutions.

This SMT is useful in situations where the JSON is not well-structured, in order to get data into Iceberg where
it can be further processed by query engines into a more manageable form. It will convert nested objects to
Maps and include Map type in the Schema. The connector will respect the Schema and create Iceberg tables with Iceberg
Map (String) columns for the JSON objects.

Note:

- You must use the `stringConverter` as the `value.converter` setting for your connector, not `jsonConverter`
- It expects JSON objects (`{...}`) in those strings.
- Message keys, tombstones, and headers are not transformed and are passed along as-is by the SMT

## Configuration

| Property | Description (default value) |
|----------------------|------------------------------------------|
| json.root | (false) Boolean value to start at root |

The `transforms.IDENTIFIER_HERE.json.root` is meant for the most inconsistent data. It will construct a Struct with a single field
called `payload` with a Schema of `Map<String, String>`.

If `transforms.IDENTIFIER_HERE.json.root` is false (the default), it will construct a Struct with inferred schemas for primitive and
array fields. Nested objects become fields of type `Map<String, String>`.

Keys with empty arrays and empty objects are filtered out from the final schema. Arrays will be typed unless the
json arrays have mixed types in which case they are converted to arrays of strings.

Example json:

```json
{
"key": 1,
"array": [1,"two",3],
"empty_obj": {},
"nested_obj": {"some_key": ["one", "two"]}
}
```

Will become the following if `json.root` is true:

```
SinkRecord.schema:
"payload" : (Optional) Map<String, String>

Sinkrecord.value (Struct):
"payload" : Map(
"key" : "1",
"array" : "[1,"two",3]"
"empty_obj": "{}"
"nested_obj": "{"some_key":["one","two"]}}"
)
```

Will become the following if `json.root` is false

```
SinkRecord.schema:
"key": (Optional) Int32,
"array": (Optional) Array<String>,
"nested_object": (Optional) Map<string, String>

SinkRecord.value (Struct):
"key" 1,
"array" ["1", "two", "3"]
"nested_object" Map ("some_key" : "["one", "two"]")
```

# KafkaMetadataTransform
_(Experimental)_

The `KafkaMetadata` injects `topic`, `partition`, `offset`, `timestamp` which are properties are the Kafka message.

## Configuration

| Property | Description (default value) |
|----------------|-----------------------------------------------------------------------------------|
| field_name | (_kafka_metadata) prefix for fields |
| nested | (false) if true, nests data on a struct else adds to top level as prefixed fields |
| external_field | (none) appends a constant `key,value` to the metadata (e.g. cluster name) |

If `nested` is on:

`_kafka_metadata.topic`, `_kafka_metadata.partition`, `_kafka_metadata.offset`, `_kafka_metadata.timestamp`

If `nested` is off:
`_kafka_metdata_topic`, `_kafka_metadata_partition`, `_kafka_metadata_offset`, `_kafka_metadata_timestamp`

# MongoDebeziumTransform
_(Experimental)_

The `MongoDebeziumTransform` SMT transforms a Mongo Debezium formatted message with `before`/`after` BSON
strings into `before`/`after` typed Structs that the `DebeziumTransform` SMT expects.

It does not (yet) support renaming columns if mongodb column is not supported by your underlying
catalog type.

## Configuration

| Property | Description |
|---------------------|--------------------------------------------------|
| array_handling_mode | `array` or `document` to set array handling mode |

Value array (the default) will encode arrays as the array datatype. It is user’s responsibility to ensure that
all elements for a given array instance are of the same type. This option is a restricting one but offers
easy processing of arrays by downstream clients.

Value document will convert the array into a struct of structs in the similar way as done by BSON serialization.
The main struct contains fields named _0, _1, _2 etc. where the name represents the index of the element in the array.
Every element is then passed as the value for the given field.
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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 org.apache.iceberg.connect.transforms;

public interface CdcConstants {

String OP_INSERT = "I";
String OP_UPDATE = "U";
String OP_DELETE = "D";

String COL_CDC = "_cdc";
String COL_OP = "op";
String COL_TS = "ts";
String COL_OFFSET = "offset";
String COL_SOURCE = "source";
String COL_TARGET = "target";
String COL_KEY = "key";
}
Loading