fqaiser94 commented on code in PR #9641:
URL: https://github.com/apache/iceberg/pull/9641#discussion_r1521874230


##########
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:
   > In general, using a schema should be strongly preferred over relying on 
inference
   
   +10
   
   I would have even preferred that we didn't support the schema-less approach 
as it's added a fair amount of complexity to the code but some users find it 
valuable so it seems to be worth it. 
   
   I'll also add that I can see some performance optimizations that we can do 
for the schema-aware code path that are likely impossible for the schema-less 
path, so that may also become a reason to prefer using schema rather than 
relying on inference long-term. 
   
   > Having a config option like "inferJsonObjectsAsMaps" (which would infer 
ALL json object fields as maps) seems like a reasonable request/feature
   > The only caveat is we would need to set the value type for the map, which 
I think we would set to string.
   
   If that's the caveat, I'm not so sure that this would be super valuable, and 
also that _seems_ like a relatively straight-forward SMT at that point? 
   
   Also a question of if it would be worth the additional code complexity to 
support (users would likely want the top-level-Map returned by 
`SinkRecord.getValue` as an `iceStruct` but any Maps inside the top-level-Map 
would want to be an `iceMap` I suspect). 
   
   Not opposed to it, just a bit iffy right now for me. 
   
   > That being said, I feel like this could be added in a follow up PR.
   
   For sure. 
   
   
   
   
   
   



-- 
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

Reply via email to