Copilot commented on code in PR #15209:
URL: https://github.com/apache/iceberg/pull/15209#discussion_r2751234070
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -81,11 +86,21 @@ class RecordConverter {
private final NameMapping nameMapping;
private final IcebergSinkConfig config;
private final Map<Integer, Map<String, NestedField>> structNameMap =
Maps.newHashMap();
+ private final int tableFormatVersion;
RecordConverter(Table table, IcebergSinkConfig config) {
this.tableSchema = table.schema();
this.nameMapping = createNameMapping(table);
this.config = config;
+
+ int formatVersion;
+ try {
+ formatVersion = TableUtil.formatVersion(table);
+ } catch (Exception ex) {
+ LOG.error("Failed to retrieve format version from table {}",
table.name(), ex);
Review Comment:
The error message should clarify the consequences of the failure. Consider
adding what will happen next (e.g., "Default values will not be supported") to
help users understand the impact.
```suggestion
LOG.error(
"Failed to retrieve format version from table {}. "
+ "Table format version will be set to -1 and some Iceberg
features may be disabled "
+ "or behave differently for this sink.",
table.name(),
ex);
```
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -385,7 +415,13 @@ protected String convertString(Object value) {
return MAPPER.writeValueAsString(value);
} else if (value instanceof Struct) {
Struct struct = (Struct) value;
- byte[] data = config.jsonConverter().fromConnectData(null,
struct.schema(), struct);
+ byte[] data;
+ try (JsonConverter jsonConverter = config.jsonConverter()) {
Review Comment:
JsonConverter is not AutoCloseable and cannot be used in a
try-with-resources statement. This will cause a compilation error. Remove the
try-with-resources and handle the conversion directly.
```suggestion
JsonConverter jsonConverter = config.jsonConverter();
try {
```
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java:
##########
@@ -72,6 +73,11 @@ RecordWriter createWriter(String tableName, SinkRecord
sample, boolean ignoreMis
@VisibleForTesting
Table autoCreateTable(String tableName, SinkRecord sample) {
+ // Determine the format version that will be used for the table
+ int formatVersion =
+ Integer.parseInt(
+
config.autoCreateProps().getOrDefault(TableProperties.FORMAT_VERSION, "-1"));
Review Comment:
Using -1 as the default format version could cause issues downstream. If
FORMAT_VERSION is not specified in autoCreateProps, the code should use a valid
default format version (e.g., 1 or 2) or explicitly handle the -1 case to
prevent unexpected behavior in format version comparisons.
```suggestion
String formatVersionStr =
config.autoCreateProps().get(TableProperties.FORMAT_VERSION);
int formatVersion =
formatVersionStr != null
? Integer.parseInt(formatVersionStr)
: TableProperties.FORMAT_VERSION_DEFAULT;
```
##########
kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java:
##########
@@ -217,8 +232,23 @@ private GenericRecord convertToStruct(
if (schemaUpdateConsumer != null) {
String parentFieldName =
structFieldId < 0 ? null :
tableSchema.findColumnName(structFieldId);
- Type type = SchemaUtils.toIcebergType(recordField.schema(),
config);
- schemaUpdateConsumer.addColumn(parentFieldName,
recordField.name(), type);
+ Type type =
+ SchemaUtils.toIcebergType(recordField.schema(), config,
tableFormatVersion);
+ org.apache.iceberg.expressions.Literal<?> defaultLiteral =
null;
+ if (tableFormatVersion >=
IcebergSinkConfig.DEFAULT_VALUE_MIN_FORMAT_VERSION) {
+ // Extract default value from Kafka Connect schema if
present
+ Object defaultValue = recordField.schema().defaultValue();
+ if (defaultValue != null) {
+ defaultLiteral =
SchemaUtils.convertDefaultValue(defaultValue, type);
+ }
+ } else {
+ LOG.info(
+ "Format version ({}) < min format version ({})
required for default value support",
+ tableFormatVersion,
+ IcebergSinkConfig.DEFAULT_VALUE_MIN_FORMAT_VERSION);
Review Comment:
This log statement will execute for every field that doesn't have default
value support during schema evolution. Consider logging this message once per
table or moving it to a higher level to avoid log spam during high-throughput
operations.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]