mxm commented on code in PR #13340:
URL: https://github.com/apache/iceberg/pull/13340#discussion_r2154454826


##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java:
##########
@@ -142,10 +151,18 @@ private void emit(
       Schema schema,
       CompareSchemasVisitor.Result result,
       PartitionSpec spec) {
-    RowData rowData =
-        result == CompareSchemasVisitor.Result.SAME
-            ? data.rowData()
-            : RowDataEvolver.convert(data.rowData(), data.schema(), schema);
+    RowData rowData;
+    if (result == CompareSchemasVisitor.Result.SAME) {
+      rowData = data.rowData();
+    } else {
+      RowDataConverter rowDataConverter =
+          converterCache.get(
+              data.schema(),
+              dataSchema ->
+                  new RowDataConverter(
+                      FlinkSchemaUtil.convert(dataSchema), 
FlinkSchemaUtil.convert(schema)));

Review Comment:
   Have we measured which conversion steps take the most time? Would it suffice 
to simply cache source and target schema while retaining the static conversion 
code? My gut feeling is that the schema conversion is the most expensive. Apart 
from caching the schema, the code here creates a series of objects, which adds 
to the memory footprint. 



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java:
##########
@@ -31,10 +33,14 @@
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.sink.dynamic.convert.RowDataConverter;
 
 @Internal
 class DynamicRecordProcessor<T> extends ProcessFunction<T, 
DynamicRecordInternal>
     implements Collector<DynamicRecord> {
+  private static final int ROW_DATA_CONVERTER_CACHE_MAXIMUM_SIZE = 1000;

Review Comment:
   This should be configurable, similarly to the other caches.



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/convert/RowDataConverter.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.flink.sink.dynamic.convert;
+
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+
+/**
+ * {@link RowDataConverter} is responsible to change the input {@link RowData} 
to make it compatible
+ * with the target schema. This is done when
+ *
+ * <ol>
+ *   <li>The input schema has fewer fields than the target schema.
+ *   <li>The table types are wider than the input type.
+ *   <li>The field order differs for source and target schema.
+ * </ol>
+ *
+ * <p>The resolution is as follows:
+ *
+ * <ol>
+ *   <li>In the first case, we would add a null values for the missing field 
(if the field is
+ *       optional).
+ *   <li>In the second case, we would convert the data for the input field to 
a wider type, e.g. int
+ *       (input type) => long (table type).
+ *   <li>In the third case, we would rearrange the input data to match the 
target table.
+ * </ol>
+ */
+public class RowDataConverter implements DataConverter {
+  private final RowData.FieldGetter[] fieldGetters;
+  private final DataConverter[] dataConverters;

Review Comment:
   I don't quite understand why we need to break apart RowDataEvolver. Could we 
simply add a cache in RowDataEvolver? I don't think the quasi code generation 
here leads to much performance gain, apart from adding to the memory footprint.



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/convert/MapConverter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.sink.dynamic.convert;
+
+import java.util.Map;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class MapConverter implements DataConverter {
+  private final ArrayData.ElementGetter keyGetter;
+  private final ArrayData.ElementGetter valueGetter;
+  private final DataConverter keyConverter;
+  private final DataConverter valueConverter;
+
+  public MapConverter(MapType sourceType, MapType targetType) {

Review Comment:
   Do we need separate classes we instantiate for every schema?



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java:
##########
@@ -69,7 +71,10 @@ public DynamicRecordInternal map(DynamicRecordInternal data) 
throws Exception {
     data.setSpec(newData.f2);
 
     if (newData.f1 == CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED) {
-      RowData newRowData = RowDataEvolver.convert(data.rowData(), 
data.schema(), newData.f0);
+      RowData newRowData =
+          new RowDataConverter(
+                  FlinkSchemaUtil.convert(data.schema()), 
FlinkSchemaUtil.convert(newData.f0))
+              .convert(data.rowData());

Review Comment:
   There is no caching here, or is there?



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