talatuyarer commented on code in PR #15596:
URL: https://github.com/apache/iceberg/pull/15596#discussion_r3212609700


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java:
##########


Review Comment:
   You can built once per write
   ```suggestion
         UnaryOperator<D> copier = copierFactory.apply(engineSchema); 
         return new BufferedFileAppender<>(
   ```



##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java:
##########
@@ -283,7 +283,7 @@ private FileAppender<D> buildShreddedAppender() {
               throw new UncheckedIOException("Failed to create shredded 
variant writer", e);
             }
           },
-          copyFunc);
+          datum -> copyFunc.apply(datum, engineSchema));

Review Comment:
   ```suggestion
             copier);
   ```



##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java:
##########
@@ -51,7 +51,7 @@ public class ParquetFormatModel<D, S, R>
     extends BaseFormatModel<D, S, ParquetValueWriter<?>, R, MessageType> {
   private final boolean isBatchReader;
   private final VariantShreddingAnalyzer<D, S> variantAnalyzer;
-  private final UnaryOperator<D> copyFunc;
+  private final BiFunction<D, S, D> copyFunc;

Review Comment:
   BiFunction<D, S, D> works, but I think a factory better matches.
   ```suggestion
   private final Function<S, UnaryOperator<D>> copierFactory;
   ```



##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java:
##########
@@ -33,7 +34,9 @@ public static void register() {
             RowType.class,
             FlinkParquetWriters::buildWriter,
             (icebergSchema, fileSchema, engineSchema, idToConstant) ->
-                FlinkParquetReaders.buildReader(icebergSchema, fileSchema, 
idToConstant)));
+                FlinkParquetReaders.buildReader(icebergSchema, fileSchema, 
idToConstant),
+            new FlinkVariantShreddingAnalyzer(),
+            (row, rowType) -> new RowDataSerializer(rowType).copy(row)));

Review Comment:
   With factory Pattern we call one serializer per write, not per row
   ```suggestion
                 rowType -> {
       RowDataSerializer serializer = new RowDataSerializer(rowType);
       return serializer::copy;
     } );
   ```



##########
spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java:
##########
@@ -53,7 +53,7 @@ public static void register() {
             (icebergSchema, fileSchema, engineSchema, idToConstant) ->
                 SparkParquetReaders.buildReader(icebergSchema, fileSchema, 
idToConstant),
             new SparkVariantShreddingAnalyzer(),
-            InternalRow::copy));
+            (internalRow, structType) -> internalRow.copy()));

Review Comment:
   And also we dont need to change this if we use FactoryPattern 



##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java:
##########
@@ -33,7 +34,9 @@ public static void register() {
             RowType.class,
             FlinkParquetWriters::buildWriter,
             (icebergSchema, fileSchema, engineSchema, idToConstant) ->
-                FlinkParquetReaders.buildReader(icebergSchema, fileSchema, 
idToConstant)));
+                FlinkParquetReaders.buildReader(icebergSchema, fileSchema, 
idToConstant),
+            new FlinkVariantShreddingAnalyzer(),
+            (row, rowType) -> new RowDataSerializer(rowType).copy(row)));

Review Comment:
   With the current BiFunction, `(row, rowType) -> new 
RowDataSerializer(rowType).copy(row)` creates a new `RowDataSerializer` for 
every buffered row (default buffer = 100). This construction is not free, as it 
involves walking `rowType.getChildren()`, building a `TypeSerializer[]` via 
`InternalSerializers.create`, a `BinaryRowDataSerializer`, and a 
`RowData.FieldGetter[]`. Since the engine schema is fixed for the entire file, 
a factory allows us to build it once and reuse it. Using the Factory Pattern, 
we can avoid recreating the serializer for a given table schema with every 
incoming record.



##########
flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkVariantShreddingType.java:
##########
@@ -0,0 +1,844 @@
+/*
+ * 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;
+
+import static org.apache.parquet.schema.Types.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.List;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.reader.ReaderUtil;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.variants.Variant;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+class TestFlinkVariantShreddingType extends CatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private Table icebergTable;
+
+  @Parameters(name = "catalogName={0}, baseNamespace={1}")
+  protected static List<Object[]> parameters() {
+    List<Object[]> parameters = Lists.newArrayList();
+    parameters.add(new Object[] {"testhadoop", Namespace.empty()});
+    parameters.add(new Object[] {"testhadoop_basenamespace", 
Namespace.of("l0", "l1")});
+    return parameters;
+  }
+
+  @Override
+  @BeforeEach
+  public void before() {
+    super.before();
+    sql("CREATE DATABASE %s", flinkDatabase);
+    sql("USE CATALOG %s", catalogName);
+    sql("USE %s", DATABASE);
+    sql(
+        "CREATE TABLE %s (id int NOT NULL, address variant NOT NULL) with 
('write.format.default'='%s','format-version'='3','parquet-shred-variants'='true','variant-inference-buffer-size'='10')",
+        TABLE_NAME, FileFormat.PARQUET.name());
+    icebergTable = 
validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME));
+  }
+
+  @TestTemplate
+  public void testExcludingNullValue() throws IOException {
+    String values =
+        "(1, parse_json('{\"name\": \"Alice\", \"age\": 30, \"dummy\": 
null}')),"
+            + " (2, parse_json('{\"name\": \"Bob\", \"age\": 25}')),"
+            + " (3, parse_json('{\"name\": \"Charlie\", \"age\": 35}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType name =
+        field(
+            "name",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType age =
+        field(
+            "age",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(8, true)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(age, name));
+    MessageType expectedSchema = parquetSchema(address);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testConsistentType() throws IOException {
+    String values =
+        "(1, parse_json('{\"age\": \"25\"}')),"
+            + " (2, parse_json('{\"age\": 30}')),"
+            + " (3, parse_json('{\"age\": \"35\"}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType age =
+        field(
+            "age",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(age));
+    MessageType expectedSchema = parquetSchema(address);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testPrimitiveType() throws IOException {
+    String values = "(1, parse_json('123')), (2, parse_json('\"abc\"')), (3, 
parse_json('12'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType address =
+        variant(
+            "address",
+            2,
+            Type.Repetition.REQUIRED,
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(8, true)));
+    MessageType expectedSchema = parquetSchema(address);
+
+    assertThat(SimpleDataUtil.tableRecords(icebergTable)).hasSize(3);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testPrimitiveDecimalType() throws IOException {
+    String values =
+        "(1, parse_json('123.56')), (2, parse_json('\"abc\"')), (3, 
parse_json('12.56'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType address =
+        variant(
+            "address",
+            2,
+            Type.Repetition.REQUIRED,
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.decimalType(2, 5)));
+    MessageType expectedSchema = parquetSchema(address);
+    assertThat(SimpleDataUtil.tableRecords(icebergTable)).hasSize(3);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testBooleanType() throws IOException {
+    String values =
+        "(1, parse_json('{\"active\": true}')),"
+            + " (2, parse_json('{\"active\": false}')),"
+            + " (3, parse_json('{\"active\": true}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType active = field("active", 
shreddedPrimitive(PrimitiveType.PrimitiveTypeName.BOOLEAN));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(active));
+    MessageType expectedSchema = parquetSchema(address);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testDecimalTypeWithInconsistentScales() throws IOException {
+    String values =
+        "(1, parse_json('{\"price\": 123.456789}')),"
+            + " (2, parse_json('{\"price\": 678.90}')),"
+            + " (3, parse_json('{\"price\": 999.99}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType price =
+        field(
+            "price",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.decimalType(6, 9)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(price));
+    MessageType expectedSchema = parquetSchema(address);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testDecimalTypeWithConsistentScales() throws IOException {
+    String values =
+        "(1, parse_json('{\"price\": 123.45}')),"
+            + " (2, parse_json('{\"price\": 678.90}')),"
+            + " (3, parse_json('{\"price\": 999.99}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType price =
+        field(
+            "price",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.decimalType(2, 5)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(price));
+    MessageType expectedSchema = parquetSchema(address);
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testArrayType() throws IOException {
+    String values =
+        "(1, parse_json('[\"java\", \"scala\", \"python\"]')),"
+            + " (2, parse_json('[\"rust\", \"go\"]')),"
+            + " (3, parse_json('[\"javascript\"]'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType arr =
+        list(
+            element(
+                shreddedPrimitive(
+                    PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType())));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, arr);
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testNestedArrayType() throws IOException {
+
+    String values =
+        "(1, parse_json('{\"tags\": [\"java\", \"scala\", \"python\"]}')),"
+            + " (2, parse_json('{\"tags\": [\"rust\", \"go\"]}')),"
+            + " (3, parse_json('{\"tags\": [\"javascript\"]}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType tags =
+        field(
+            "tags",
+            list(
+                element(
+                    shreddedPrimitive(
+                        PrimitiveType.PrimitiveTypeName.BINARY,
+                        LogicalTypeAnnotation.stringType()))));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(tags));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testNestedObjectType() throws IOException {
+
+    String values =
+        "(1, parse_json('{\"location\": {\"city\": \"Seattle\", \"zip\": 
98101}, \"tags\": [\"java\", \"scala\", \"python\"]}')),"
+            + " (2, parse_json('{\"location\": {\"city\": \"Portland\", 
\"zip\": 97201}}')),"
+            + " (3, parse_json('{\"location\": {\"city\": \"NYC\", \"zip\": 
10001}}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType city =
+        field(
+            "city",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType zip =
+        field(
+            "zip",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(32, true)));
+    GroupType location = field("location", objectFields(city, zip));
+    GroupType tags =
+        field(
+            "tags",
+            list(
+                element(
+                    shreddedPrimitive(
+                        PrimitiveType.PrimitiveTypeName.BINARY,
+                        LogicalTypeAnnotation.stringType()))));
+
+    GroupType address =
+        variant("address", 2, Type.Repetition.REQUIRED, objectFields(location, 
tags));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testLazyInitializationWithBufferedRows() throws IOException {
+
+    String values =
+        "(1, parse_json('{\"name\": \"Alice\", \"age\": 30}')),"
+            + " (2, parse_json('{\"name\": \"Bob\", \"age\": 25}')),"
+            + " (3, parse_json('{\"name\": \"Charlie\", \"age\": 35}')),"
+            + " (4, parse_json('{\"name\": \"David\", \"age\": 28}')),"
+            + " (5, parse_json('{\"name\": \"Eve\", \"age\": 32}')),"
+            + " (6, parse_json('{\"name\": \"Frank\", \"age\": 40}')),"
+            + " (7, parse_json('{\"name\": \"Grace\", \"age\": 27}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType name =
+        field(
+            "name",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType age =
+        field(
+            "age",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(8, true)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(age, name));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+    assertThat(genericRowData()).hasSize(7);
+  }
+
+  @TestTemplate
+  public void testColumnIndexTruncateLength() throws IOException {
+    sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME);
+
+    int customTruncateLength = 10;
+    sql(
+        "ALTER TABLE %s SET ('%s'='%d')",
+        TABLE_NAME, "parquet.columnindex.truncate.length", 
customTruncateLength);
+
+    StringBuilder valuesBuilder = new StringBuilder();
+    for (int i = 1; i <= 10; i++) {
+      if (i > 1) {
+        valuesBuilder.append(", ");
+      }
+
+      String longValue = "A".repeat(20);
+      valuesBuilder.append(
+          String.format(
+              "(%d, parse_json('{\"description\": \"%s\", \"id\": %d}'))", i, 
longValue, i));
+    }
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString());
+
+    GroupType description =
+        field(
+            "description",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType id =
+        field(
+            "id",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(8, true)));
+    GroupType address =
+        variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(description, id));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+    assertThat(genericRowData()).hasSize(10);
+  }
+
+  @TestTemplate
+  public void testIntegerFamilyPromotion() throws IOException {
+
+    // Mix of INT8, INT16, INT32, INT64 - should promote to INT64
+    String values =
+        "(1, parse_json('{\"value\": 10}')),"
+            + " (2, parse_json('{\"value\": 1000}')),"
+            + " (3, parse_json('{\"value\": 100000}')),"
+            + " (4, parse_json('{\"value\": 10000000000}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType value =
+        field(
+            "value",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT64, 
LogicalTypeAnnotation.intType(64, true)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(value));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testDecimalFamilyPromotion() throws IOException {
+
+    // Test that they get promoted to the most capable decimal type observed
+    String values =
+        "(1, parse_json('{\"value\": 1.5}')),"
+            + " (2, parse_json('{\"value\": 123.456789}')),"
+            + " (3, parse_json('{\"value\": 123456789123456.789}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType value =
+        field(
+            "value",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY,
+                16,
+                LogicalTypeAnnotation.decimalType(6, 21)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(value));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testDataRoundTripWithShredding() throws IOException {
+    String values =
+        "(1, parse_json('{\"name\": \"Alice\", \"age\": 30}')),"
+            + " (2, parse_json('{\"name\": \"Bob\", \"age\": 25}')),"
+            + " (3, parse_json('{\"name\": \"Charlie\", \"age\": 35}'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType name =
+        field(
+            "name",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType age =
+        field(
+            "age",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.INT32, 
LogicalTypeAnnotation.intType(8, true)));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(age, name));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+
+    // Verify that we can read the data back correctly
+    List<Row> rows =
+        sql(
+            "SELECT id, JSON_VALUE(address, '$.name'),"
+                + " JSON_VALUE(address, '$.age' RETURNING int)"
+                + " FROM %s ORDER BY id",
+            TABLE_NAME);
+    assertThat(rows).hasSize(3);
+    assertThat(rows.get(0).getField(0)).isEqualTo(1);
+    assertThat(rows.get(0).getField(1)).isEqualTo("Alice");
+    assertThat(rows.get(0).getField(2)).isEqualTo(30);
+    assertThat(rows.get(1).getField(0)).isEqualTo(2);
+    assertThat(rows.get(1).getField(1)).isEqualTo("Bob");
+    assertThat(rows.get(1).getField(2)).isEqualTo(25);
+    assertThat(rows.get(2).getField(0)).isEqualTo(3);
+    assertThat(rows.get(2).getField(1)).isEqualTo("Charlie");
+    assertThat(rows.get(2).getField(2)).isEqualTo(35);
+  }
+
+  @TestTemplate
+  public void testVariantWithNullValues() throws IOException {
+
+    String values =
+        "(1, parse_json('null'))," + " (2, parse_json('null'))," + " (3, 
parse_json('null'))";
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, values);
+
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED);
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testArrayOfNullElementsWithShredding() throws IOException {
+
+    sql(
+        "INSERT INTO %s VALUES (1, parse_json('[null, null, null]')), "
+            + "(2, parse_json('[null]'))",
+        TABLE_NAME);
+
+    // Array elements are all null, element type is null, falls back to 
unshredded
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED);
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testInfrequentFieldPruning() throws IOException {
+    sql("ALTER TABLE %s SET('variant-inference-buffer-size'='11')", 
TABLE_NAME);
+    StringBuilder valuesBuilder = new StringBuilder();
+    for (int i = 1; i <= 11; i++) {
+      if (i > 1) {
+        valuesBuilder.append(", ");
+      }
+
+      if (i == 1) {
+        // Only the first row has rare_field
+        valuesBuilder.append(
+            String.format(
+                "(%d, parse_json('{\"name\": \"User%d\", \"rare_field\": 
\"rare\"}'))", i, i));
+      } else {
+        valuesBuilder.append(String.format("(%d, parse_json('{\"name\": 
\"User%d\"}'))", i, i));
+      }
+    }
+
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString());
+
+    // rare_field appears in 1/11 rows, should be pruned
+    // name appears in 11/11 rows and should be kept
+    GroupType name =
+        field(
+            "name",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(name));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testMixedTypeTieBreaking() throws IOException {
+    StringBuilder valuesBuilder = new StringBuilder();
+    for (int i = 1; i <= 10; i++) {
+      if (i > 1) {
+        valuesBuilder.append(", ");
+      }
+
+      if (i <= 5) {
+        valuesBuilder.append(String.format("(%d, parse_json('{\"val\": 
%d}'))", i, i));
+      } else {
+        valuesBuilder.append(String.format("(%d, parse_json('{\"val\": 
\"text%d\"}'))", i, i));
+      }
+    }
+
+    sql("INSERT INTO %s VALUES %s", TABLE_NAME, valuesBuilder.toString());
+
+    // 5 ints + 5 strings is a tie so STRING wins (higher TIE_BREAK_PRIORITY)
+    GroupType val =
+        field(
+            "val",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(val));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testFieldOnlyAfterBuffer() throws IOException {
+    getTableEnv()
+        .getConfig()
+        .getConfiguration()
+        .setString("table.exec.resource.default-parallelism", "1");
+
+    sql("ALTER TABLE %s SET('variant-inference-buffer-size'='3')", TABLE_NAME);
+
+    sql(
+        "CREATE TEMPORARY VIEW tmp_source AS "
+            + "SELECT * FROM (VALUES "
+            + "(1, parse_json('{\"name\": \"Alice\"}')), "
+            + "(2, parse_json('{\"name\": \"Bob\"}')), "
+            + "(3, parse_json('{\"name\": \"Charlie\"}')), "
+            + "(4, parse_json('{\"name\": \"David\", \"score\": 95}')), "
+            + "(5, parse_json('{\"name\": \"Eve\", \"score\": 88}')), "
+            + "(6, parse_json('{\"name\": \"Frank\", \"score\": 72}')), "
+            + "(7, parse_json('{\"name\": \"Grace\", \"score\": 91}'))"
+            + ") AS t(id, address)");
+
+    sql("INSERT INTO %s SELECT id, address FROM tmp_source ORDER BY id", 
TABLE_NAME);
+
+    // Schema is determined from buffer (rows 1-3) which only has "name".
+    // "score" is not shredded
+    GroupType name =
+        field(
+            "name",
+            shreddedPrimitive(
+                PrimitiveType.PrimitiveTypeName.BINARY, 
LogicalTypeAnnotation.stringType()));
+    GroupType address = variant("address", 2, Type.Repetition.REQUIRED, 
objectFields(name));
+    MessageType expectedSchema = parquetSchema(address);
+
+    verifyParquetSchema(icebergTable, expectedSchema);
+
+    // Verify all data round-trips despite "score" not being shredded
+    List<Row> rows =
+        sql(
+            "SELECT id, JSON_VALUE(address, '$.name'),"
+                + " JSON_VALUE(address, '$.score' returning int)"
+                + " FROM %s ORDER BY id",
+            TABLE_NAME);
+    assertThat(rows).hasSize(7);
+    assertThat(rows.get(0).getField(1)).isEqualTo("Alice");
+    assertThat(rows.get(0).getField(2)).isNull();
+    assertThat(rows.get(3).getField(1)).isEqualTo("David");
+    assertThat(rows.get(3).getField(2)).isEqualTo(95);
+    assertThat(rows.get(6).getField(1)).isEqualTo("Grace");
+    assertThat(rows.get(6).getField(2)).isEqualTo(91);
+
+    getTableEnv()

Review Comment:
   nit: This mutates table.exec.resource.default-parallelism to "1" and resets 
to "4" only at the end. If any assertion throws, the reset never runs and 
downstream tests inherit the override. I believe you can use @AfterEach.



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

Reply via email to