Skip to content

Commit 9bd04b5

Browse files
authored
Add option to change case of destination/iceberg table names (#389)
* Add option to change case of destination/iceberg table names * Add option to change case of destination/iceberg table names * Add option to change case of destination/iceberg table names * Add option to change case of destination/iceberg table names * Add option to change case of destination/iceberg table names * Add option to change case of destination/iceberg table names --------- Co-authored-by: Ismail Simsek <ismail.simsek@verivox.com>
1 parent 623377d commit 9bd04b5

File tree

3 files changed

+25
-4
lines changed

3 files changed

+25
-4
lines changed

debezium-server-iceberg-sink/src/main/java/io/debezium/server/iceberg/IcebergChangeConsumer.java

Lines changed: 14 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,10 @@ public class IcebergChangeConsumer extends BaseChangeConsumer implements Debeziu
8484
protected Optional<String> destinationRegexp;
8585
@ConfigProperty(name = "debezium.sink.iceberg.destination-regexp-replace", defaultValue = "")
8686
protected Optional<String> destinationRegexpReplace;
87+
@ConfigProperty(name = "debezium.sink.iceberg.destination-uppercase-table-names", defaultValue = "false")
88+
protected boolean destinationUppercaseTableNames;
89+
@ConfigProperty(name = "debezium.sink.iceberg.destination-lowercase-table-names", defaultValue = "false")
90+
protected boolean destinationLowercaseTableNames;
8791
@ConfigProperty(name = "debezium.sink.iceberg.table-prefix", defaultValue = "")
8892
Optional<String> tablePrefix;
8993
@ConfigProperty(name = "debezium.sink.iceberg.table-namespace", defaultValue = "default")
@@ -144,7 +148,7 @@ public void handleBatch(List<ChangeEvent<Object, Object>> records, DebeziumEngin
144148
Map<String, List<IcebergChangeEvent>> result =
145149
records.stream()
146150
.map((ChangeEvent<Object, Object> e)
147-
-> new IcebergChangeEvent(e.destination(), getBytes(e.value()), e.key() == null ? null : getBytes(e.key())))
151+
-> new IcebergChangeEvent(e.destination(), getBytes(e.value()), e.key() == null ? null : getBytes(e.key())))
148152
.collect(Collectors.groupingBy(IcebergChangeEvent::destination));
149153

150154
// consume list of events for each destination table
@@ -178,8 +182,8 @@ public Table loadIcebergTable(TableIdentifier tableId, IcebergChangeEvent sample
178182
}
179183
try {
180184
return IcebergUtil.createIcebergTable(icebergCatalog, tableId, sampleEvent.icebergSchema(createIdentifierFields), writeFormat);
181-
} catch (Exception e){
182-
throw new DebeziumException("Failed to create table from debezium event schema:"+tableId+" Error:" + e.getMessage(), e);
185+
} catch (Exception e) {
186+
throw new DebeziumException("Failed to create table from debezium event schema:" + tableId + " Error:" + e.getMessage(), e);
183187
}
184188
});
185189
}
@@ -204,6 +208,12 @@ public TableIdentifier mapDestination(String destination) {
204208
.replaceAll(destinationRegexp.orElse(""), destinationRegexpReplace.orElse(""))
205209
.replace(".", "_");
206210

207-
return TableIdentifier.of(Namespace.of(namespace), tablePrefix.orElse("") + tableName);
211+
if (destinationUppercaseTableNames) {
212+
return TableIdentifier.of(Namespace.of(namespace), (tablePrefix.orElse("") + tableName).toUpperCase());
213+
} else if (destinationLowercaseTableNames) {
214+
return TableIdentifier.of(Namespace.of(namespace), (tablePrefix.orElse("") + tableName).toLowerCase());
215+
} else {
216+
return TableIdentifier.of(Namespace.of(namespace), tablePrefix.orElse("") + tableName);
217+
}
208218
}
209219
}

debezium-server-iceberg-sink/src/test/java/io/debezium/server/iceberg/IcebergChangeConsumerTest.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -334,6 +334,15 @@ public void testPartitionedTable() {
334334
public void testMapDestination() {
335335
assertEquals(TableIdentifier.of(Namespace.of(namespace), "debeziumcdc_table"), icebergConsumer.mapDestination("table1"));
336336
assertEquals(TableIdentifier.of(Namespace.of(namespace), "debeziumcdc_table"), icebergConsumer.mapDestination("table2"));
337+
icebergConsumer.destinationUppercaseTableNames = true;
338+
icebergConsumer.destinationLowercaseTableNames = false;
339+
assertEquals(TableIdentifier.of(Namespace.of(namespace), "DEBEZIUMCDC_TABLE_NAME"), icebergConsumer.mapDestination("table_name"));
340+
assertEquals(TableIdentifier.of(Namespace.of(namespace), "DEBEZIUMCDC_TABLE_NAME"), icebergConsumer.mapDestination("Table_Name"));
341+
assertEquals(TableIdentifier.of(Namespace.of(namespace), "DEBEZIUMCDC_TABLE_NAME"), icebergConsumer.mapDestination("TABLE_NAME"));
342+
icebergConsumer.destinationUppercaseTableNames = false;
343+
icebergConsumer.destinationLowercaseTableNames = true;
344+
assertEquals(TableIdentifier.of(Namespace.of(namespace), "debeziumcdc_table_name"), icebergConsumer.mapDestination("Table_Name"));
345+
assertEquals(TableIdentifier.of(Namespace.of(namespace), "debeziumcdc_table_name"), icebergConsumer.mapDestination("TABLE_NAME"));
337346
}
338347

339348
public static class TestProfile implements QuarkusTestProfile {

docs/DOCS.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,8 @@ tables created automatically with the first start.
2929
| `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. |
3030
| `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`. |
3131
| `debezium.sink.iceberg.destination-regexp-replace` | `` | Regexp replace part to modify destination iceberg table name |
32+
| `debezium.sink.iceberg.destination-uppercase-table-names` | `false` | Maps debezium destination to uppercase iceberg table names |
33+
| `debezium.sink.iceberg.destination-lowercase-table-names` | `false` | Maps debezium destination to lowercase iceberg table names |
3234
| `debezium.sink.batch.batch-size-wait` | `NoBatchSizeWait` | Batch size wait strategy, Used to optimize data file size and upload interval. explained below. |
3335
| `debezium.sink.iceberg.{iceberg.prop.name}` | | [Iceberg config](https://iceberg.apache.org/docs/latest/configuration/) this settings are passed to Iceberg (without the prefix) |
3436
| `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](https://debezium.io/documentation/reference/stable/development/engine.html#advanced-consuming) |

0 commit comments

Comments
 (0)