Replicates database CDC events to Iceberg tables(Cloud storage, hdfs) without using Spark, Kafka or Streaming platform in between.
Iceberg consumer replicates database CDC events to destination Iceberg tables. It is possible to replicate source
data with upsert or append modes.
When event and key schema enabled (debezium.format.value.schemas.enable=true
, debezium.format.key.schemas.enable=true
) destination Iceberg
tables created automatically with the first start.
Config | Default | Description |
---|---|---|
debezium.sink.iceberg.warehouse |
Root path of the Iceberg data warehouse | |
debezium.sink.iceberg.catalog-name |
default |
User-specified Iceberg catalog name. |
debezium.sink.iceberg.table-namespace |
default |
A namespace in the catalog. ex: SELECT * FROM prod.db.table -- catalog: prod, namespace: db, table: table |
debezium.sink.iceberg.table-prefix |
`` | Iceberg table name prefix, prefix added to iceberg table names. |
debezium.sink.iceberg.write.format.default |
parquet |
Default file format for the table; parquet , avro , or orc |
debezium.sink.iceberg.allow-field-addition |
true |
Allow field addition to target tables. Enables automatic schema expansion. |
debezium.sink.iceberg.upsert |
true |
Running consumer in upsert mode, overwriting updated rows. explained below. |
debezium.sink.iceberg.upsert-keep-deletes |
true |
When running with upsert mode, keeps deleted rows in target table (soft delete). |
debezium.sink.iceberg.upsert-dedup-column |
__source_ts_ms |
With upsert mode used to deduplicate data. row with highest __source_ts_ms kept(last change event). dont change! |
debezium.sink.iceberg.create-identifier-fields |
true |
When set to false the consumer will create tables without identifier fields. useful when user wants to consume nested events with append only mode. |
debezium.sink.iceberg.destination-regexp |
`` | Regexp to modify destination iceberg table name. For example with this setting, its possible to combine some tables table_ptt1 ,table_ptt2 to one table_combined . |
debezium.sink.iceberg.destination-regexp-replace |
`` | Regexp replace part to modify destination iceberg table name |
debezium.sink.iceberg.destination-uppercase-table-names |
false |
Maps debezium destination to uppercase iceberg table names |
debezium.sink.iceberg.destination-lowercase-table-names |
false |
Maps debezium destination to lowercase iceberg table names |
debezium.sink.batch.batch-size-wait |
NoBatchSizeWait |
Batch size wait strategy, Used to optimize data file size and upload interval. explained below. |
debezium.sink.iceberg.{iceberg.prop.name} |
Iceberg config this settings are passed to Iceberg (without the prefix) | |
debezium.source.offset.storage |
io.debezium.server.iceberg.offset.IcebergOffsetBackingStore |
The name of the Java class that is responsible for persistence of connector offsets. see debezium doc |
debezium.source.offset.storage.iceberg.table-name |
debezium_offset_storage |
Destination table name to store connector offsets. |
debezium.source.schema.history.internal |
io.debezium.server.iceberg.history.IcebergSchemaHistory |
The name of the Java class that is responsible for persistence of the database schema history. see debezium doc |
debezium.source.schema.history.internal.iceberg.table-name |
debezium_schema_history_storage |
Destination table name to store database schema history. |
By default, Iceberg consumer is running with upsert mode debezium.sink.iceberg.upsert=true
.
Upsert mode uses source Primary Key and does upsert on target table(delete followed by insert). For the tables without
Primary Key consumer falls back to append mode.
With upsert mode data deduplication is done. Deduplication is done based on __source_ts_ms
value and event type __op
.
its is possible to change this field using debezium.sink.iceberg.upsert-dedup-column=__source_ts_ms
(Currently only
Long field type supported.)
Operation type priorities are {"c":1, "r":2, "u":3, "d":4}
. When two records with same key and same __source_ts_ms
values received then the record with higher __op
priority is kept and added to destination table and duplicate record
is dropped from the batch.
By default debezium.sink.iceberg.upsert-keep-deletes=true
keeps deletes in the Iceberg table, setting it to false
will remove deleted records from the destination Iceberg table too. With this config it's possible to keep last version
of a
deleted record in the destination Iceberg table(doing soft delete for this records __deleted
is set to true
).
Setting debezium.sink.iceberg.upsert=false
will set the operation mode to append. With append mode data deduplication
is not done and all received records are appended to destination table.
Note: For the tables without primary key operation mode falls back to append even upsert mode is used.
Debezium extracts database events in real time and this could cause too frequent commits and too many small files. Which is not optimal for performance especially when near realtime data feed is sufficient. To avoid this problem following batch-size-wait classes are available to adjust batch size and interval.
Batch size wait adds delay between consumer calls to increase total number of events consumed per call. Meanwhile,
events are collected in memory.
This setting should be configured together with debezium.source.max.queue.size
and debezium.source.max.batch.size
debezium properties
This is default configuration by default consumer will not use any wait. All the events are consumed immediately.
MaxBatchSizeWait uses debezium metrics to optimize batch size.
MaxBatchSizeWait periodically checks streaming queue size and waits until it reaches to max.batch.size
.
Maximum wait and check intervals are controlled
by debezium.sink.batch.batch-size-wait.max-wait-ms
, debezium.sink.batch.batch-size-wait.wait-interval-ms
properties.
example setup to receive 2048 events per commit. maximum wait is set to 30 seconds, streaming queue current size checked every 5 seconds
debezium.sink.batch.batch-size-wait=MaxBatchSizeWait
debezium.source.connector.class=io.debezium.connector.postgresql.PostgresConnector
debezium.source.max.batch.size=2048
debezium.source.max.queue.size=16000
debezium.sink.batch.batch-size-wait.max-wait-ms=30000
debezium.sink.batch.batch-size-wait.wait-interval-ms=5000
Iceberg tables are named by following rule : table-namespace
.table-prefix``database.server.name
database
table
For example:
debezium.sink.iceberg.table-namespace=default
database.server.name=testc
debezium.sink.iceberg.table-prefix=cdc_
With above config database table = inventory.customers
is replicated to default.testc_cdc_inventory_customers
This implementation saves CDC offset to an iceberg table. Debezium keeps source offset to track binlog position.
debezium.source.offset.storage=io.debezium.server.iceberg.offset.IcebergOffsetBackingStore
debezium.source.offset.storage.iceberg.table-name=debezium_offset_storage_table
This implementation saves database history to an iceberg table.
debezium.source.database.history=io.debezium.server.iceberg.history.IcebergSchemaHistory
debezium.source.database.history.iceberg.table-name=debezium_database_history_storage_table
For best experience its recommended to run consumer with event flattening. For further details
on Message transformations
please
see debezium doc
Example Event flattening setting:
debezium.transforms=unwrap
debezium.transforms.unwrap.type=io.debezium.transforms.ExtractNewRecordState
debezium.transforms.unwrap.add.fields=op,table,source.ts_ms,db
debezium.transforms.unwrap.add.headers=db
debezium.transforms.unwrap.delete.handling.mode=rewrite
Without event flattening iceberg consumer can only run with append mode. Without event flattening upsert mode and creation of identifier fields are not supported.
Settings for running consumer without event flattening:
debezium.sink.iceberg.upsert=false
debezium.sink.iceberg.create-identifier-fields=false
All the properties starting with debezium.sink.iceberg.__ICEBERG_CONFIG__
are passed to Iceberg, and to hadoopConf
debezium.sink.iceberg.{iceberg.prop.name}=xyz-value # passed to iceberg!
Read application.properties.example
Source systems frequently undergo schema changes. This can include adding new fields, removing existing ones, or modifying the structure of existing fields. Here, we'll document the potential schema changes we anticipate and how the system currently handles them.
NOTE: Full schema evaluation is not supported. But sema expansion like field addition, data type expansion are supported,
see debezium.sink.iceberg.allow-field-addition
setting.
New columns in the source data are not automatically reflected in the destination Iceberg table. This means data for these new columns will be ignored until the corresponding column is manually added to the destination table schema.
new columns are automatically added to destination table and they are populated with new data. This is automatically done by consumer.
After removal, these column values are populated with null value. columns are kept in the destination table, no change applied to destination table.
This is combination of above two cases : Old column will be populated with null values and new column will be populated when added to iceberg table(it is either added automatically by consumer or added manually by user)
This is the scenario when source field type changes. support for this kind of changes is limited. Only safe data type expansions are supported forexample converting int to long is supported but converting deciman to int is not supported.
In this case consumer will try to change destination data type automatically. For incompatible changes consumer will throw exception. For example float to integer conversion is not supported but int to double conversion is supported.
In this case consumer will convert source field value to destination type value using jackson. Conversion is done by jackson If representation cannot be converted to destination type then default value is returned by jackson!
for example this is conversion rule for Long type:
Method that will try to convert value of this node to a Java long. Numbers are coerced using default Java rules; booleans convert to 0 (false) and 1 (true), and Strings are parsed using default Java language integer parsing rules.
If representation cannot be converted to a long (including structured types like Objects and Arrays), default value of 0 will be returned; no exceptions are thrown.
for example this is conversion rule for boolean type:
Method that will try to convert value of this node to a Java boolean. JSON booleans map naturally; integer numbers other than 0 map to true, and 0 maps to false and Strings 'true' and 'false' map to corresponding values.
If representation can not be converted to a boolean value (including structured types like Objects and Arrays), specified defaultValue will be returned; no exceptions are thrown.
This consumer appends all CDC events to single Iceberg table as json string.
This table partitioned by event_destination,event_sink_timestamptz
debezium.sink.type=icebergevents
debezium.sink.iceberg.catalog-name=default
Iceberg table definition:
static final String TABLE_NAME = "debezium_events";
static final Schema TABLE_SCHEMA = new Schema(
required(1, "event_destination", Types.StringType.get()),
optional(2, "event_key", Types.StringType.get()),
optional(3, "event_value", Types.StringType.get()),
optional(4, "event_sink_epoch_ms", Types.LongType.get()),
optional(5, "event_sink_timestamptz", Types.TimestampType.withZone())
);
static final PartitionSpec TABLE_PARTITION = PartitionSpec.builderFor(TABLE_SCHEMA)
.identity("event_destination")
.hour("event_sink_timestamptz")
.build();
static final SortOrder TABLE_SORT_ORDER = SortOrder.builderFor(TABLE_SCHEMA)
.asc("event_sink_epoch_ms", NullOrder.NULLS_LAST)
.build();