fqaiser94 commented on code in PR #9641: URL: https://github.com/apache/iceberg/pull/9641#discussion_r1515260098
########## kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java: ########## @@ -0,0 +1,508 @@ +/* + * 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.data; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.DateTimeParseException; +import java.time.temporal.Temporal; +import java.util.Base64; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.kafka.connect.data.Struct; + +class RecordConverter { + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final DateTimeFormatter OFFSET_TIMESTAMP_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HHmm", "Z") + .toFormatter(); + + private final Schema tableSchema; + private final NameMapping nameMapping; + private final IcebergSinkConfig config; + private final Map<Integer, Map<String, NestedField>> structNameMap = Maps.newHashMap(); + + RecordConverter(Table table, IcebergSinkConfig config) { + this.tableSchema = table.schema(); + this.nameMapping = createNameMapping(table); + this.config = config; + } + + Record convert(Object data) { + return convert(data, null); + } + + Record convert(Object data, SchemaUpdate.Consumer schemaUpdateConsumer) { + if (data instanceof Struct || data instanceof Map) { + return convertStructValue(data, tableSchema.asStruct(), -1, schemaUpdateConsumer); + } + throw new UnsupportedOperationException("Cannot convert type: " + data.getClass().getName()); + } + + private NameMapping createNameMapping(Table table) { + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + } + + private Object convertValue( + Object value, Type type, int fieldId, SchemaUpdate.Consumer schemaUpdateConsumer) { + if (value == null) { + return null; + } + switch (type.typeId()) { + case STRUCT: + return convertStructValue(value, type.asStructType(), fieldId, schemaUpdateConsumer); + case LIST: + return convertListValue(value, type.asListType(), schemaUpdateConsumer); + case MAP: + return convertMapValue(value, type.asMapType(), schemaUpdateConsumer); + case INTEGER: + return convertInt(value); + case LONG: + return convertLong(value); + case FLOAT: + return convertFloat(value); + case DOUBLE: + return convertDouble(value); + case DECIMAL: + return convertDecimal(value, (DecimalType) type); + case BOOLEAN: + return convertBoolean(value); + case STRING: + return convertString(value); + case UUID: + return convertUUID(value); + case BINARY: + case FIXED: + return convertBase64Binary(value); + case DATE: + return convertDateValue(value); + case TIME: + return convertTimeValue(value); + case TIMESTAMP: + return convertTimestampValue(value, (TimestampType) type); + } + throw new UnsupportedOperationException("Unsupported type: " + type.typeId()); + } + + protected GenericRecord convertStructValue( + Object value, + StructType schema, + int parentFieldId, + SchemaUpdate.Consumer schemaUpdateConsumer) { + if (value instanceof Map) { + return convertToStruct((Map<?, ?>) value, schema, parentFieldId, schemaUpdateConsumer); + } else if (value instanceof Struct) { + return convertToStruct((Struct) value, schema, parentFieldId, schemaUpdateConsumer); + } + throw new IllegalArgumentException("Cannot convert to struct: " + value.getClass().getName()); + } + + private GenericRecord convertToStruct( + Map<?, ?> map, + StructType schema, + int structFieldId, + SchemaUpdate.Consumer schemaUpdateConsumer) { + GenericRecord result = GenericRecord.create(schema); + map.forEach( + (recordFieldNameObj, recordFieldValue) -> { + String recordFieldName = recordFieldNameObj.toString(); + NestedField tableField = lookupStructField(recordFieldName, schema, structFieldId); + if (tableField == null) { + // add the column if schema evolution is on, otherwise skip the value, + // skip the add column if we can't infer the type + if (schemaUpdateConsumer != null) { + Type type = SchemaUtils.inferIcebergType(recordFieldValue, config); Review Comment: CMIIW but I see a little "discrepancy" here. When we auto create tables, [we prefer to infer the type](https://github.com/apache/iceberg/blob/f4ba90d648d1ec692134c43fc24fe389aa687d5e/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java#L80) based on the `SinkRecord.valueSchema()`, which I think is good. When we evolve tables however, we infer the iceberg type based purely on the java type of the object. It would be better if we did similar logic here as the auto-create-table-code-path and gave preference to the kafka-connect-type before falling back to the java-type of the object when trying to infer the iceberg type . One case where this would make a difference I think is in the case of Map type objects. Currently, I think that if the `SinkRecord.valueSchema()` specifies a Map type for a field, we will honour that when creating a table and create the field with an iceberg MapType (relevant code [here](https://github.com/apache/iceberg/blob/19cf37edec52a39befcaa443feb5e629bd87a28e/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java#L266)) However, if a new SinkRecord comes in for the same table, and this record includes a new Map field, we will evolve the table's schema to include a new StructType field instead. -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org