Skip to content
Open
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,11 @@
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.sink.SinkRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

class IcebergWriter implements RecordWriter {
private static final Logger LOG = LoggerFactory.getLogger(IcebergWriter.class);
private final Table table;
private final TableReference tableReference;
private final IcebergSinkConfig config;
Expand Down Expand Up @@ -77,24 +80,43 @@ public void write(SinkRecord record) {

private Record convertToRow(SinkRecord record) {
if (!config.evolveSchemaEnabled()) {
return recordConverter.convert(record.value());
return recordConverter.convert(record);
}

SchemaUpdate.Consumer updates = new SchemaUpdate.Consumer();
Record row = recordConverter.convert(record.value(), updates);
recordConverter.evolveSchema(record, updates);

if (!updates.empty()) {
// complete the current file
flush();
// apply the schema updates, this will refresh the table
SchemaUtils.applySchemaUpdates(table, updates);
// initialize a new writer with the new schema
initNewWriter();
// convert the row again, this time using the new table schema
row = recordConverter.convert(record.value(), null);
try {
// complete the current file
flush();

// apply the schema updates, this will refresh the table
SchemaUtils.applySchemaUpdates(table, updates);
LOG.info(
"Schema evolution on table {} caused by record at topic: {}, partition: {}, offset: {}",
table.name(),
record.topic(),
record.kafkaPartition(),
record.kafkaOffset());
} catch (Exception e) {
LOG.error(
"Schema updates for table {} not applied by record at topic: {}, partition: {}, offset: {} because {}. Data will still be written to table",
table.name(),
record.topic(),
record.kafkaPartition(),
record.kafkaOffset(),
e.getMessage(),
e);
} finally {
// initialize a new writer with the latest schema - in case any other task has already
// applied the schema updates
initNewWriter();
}
}

return row;
// convert the row, this time new table schema will be used
return recordConverter.convert(record);
}

private void flush() {
Expand Down
Loading