steveloughran commented on code in PR #14297:
URL: https://github.com/apache/iceberg/pull/14297#discussion_r2983740823


##########
spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/variant/TestVariantShredding.java:
##########
@@ -0,0 +1,1015 @@
+/*
+ * 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.spark.variant;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
+import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
+import static org.apache.parquet.schema.Types.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.CatalogTestBase;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.types.Types;
+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.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestVariantShredding extends CatalogTestBase {
+
+  private static final Schema SCHEMA =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "address", Types.VariantType.get()));
+
+  private static final Schema SCHEMA2 =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "address", Types.VariantType.get()),
+          Types.NestedField.optional(3, "metadata", Types.VariantType.get()));
+
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  protected static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HADOOP.catalogName(),
+        SparkCatalogConfig.HADOOP.implementation(),
+        SparkCatalogConfig.HADOOP.properties()
+      },
+    };
+  }
+
+  @BeforeAll
+  public static void startMetastoreAndSpark() {
+    // First call parent to initialize metastore and spark with local[2]
+    CatalogTestBase.startMetastoreAndSpark();
+
+    // Now stop and recreate spark with local[1] to write all rows to a single 
file
+    if (spark != null) {
+      spark.stop();
+    }
+
+    spark =
+        SparkSession.builder()
+            .master("local[1]") // Use one thread to write the rows to a 
single parquet file
+            .config("spark.driver.host", 
InetAddress.getLoopbackAddress().getHostAddress())
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config("spark.hadoop." + METASTOREURIS.varname, 
hiveConf.get(METASTOREURIS.varname))
+            
.config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+  }
+
+  @BeforeEach
+  public void before() {
+    super.before();
+    validationCatalog.createTable(
+        tableIdent, SCHEMA, null, Map.of(TableProperties.FORMAT_VERSION, "3"));
+  }
+
+  @AfterEach
+  public void after() {
+    spark.conf().unset(SparkSQLProperties.SHRED_VARIANTS);
+    spark.conf().unset(SparkSQLProperties.VARIANT_INFERENCE_BUFFER_SIZE);
+    validationCatalog.dropTable(tableIdent, true);
+  }
+
+  @TestTemplate
+  public void testVariantShreddingDisabled() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "false");
+
+    String values = "(1, parse_json('{\"city\": \"NYC\", \"zip\": 10001}')), 
(2, null)";
+    sql("INSERT INTO %s VALUES %s", tableName, values);
+
+    GroupType address = variant("address", 2, Type.Repetition.OPTIONAL);
+    MessageType expectedSchema = parquetSchema(address);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    verifyParquetSchema(table, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testExcludingNullValue() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "true");
+
+    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", tableName, 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);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    verifyParquetSchema(table, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testInconsistentType() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "true");
+
+    String values =
+        "(1, parse_json('{\"age\": \"25\"}')),"

Review Comment:
   java17 has that """" multiline string thing which is ideal for json like 
this 



##########
spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/variant/TestVariantShredding.java:
##########
@@ -0,0 +1,1015 @@
+/*
+ * 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.spark.variant;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
+import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
+import static org.apache.parquet.schema.Types.optional;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.CatalogTestBase;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.types.Types;
+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.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestVariantShredding extends CatalogTestBase {
+
+  private static final Schema SCHEMA =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "address", Types.VariantType.get()));
+
+  private static final Schema SCHEMA2 =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "address", Types.VariantType.get()),
+          Types.NestedField.optional(3, "metadata", Types.VariantType.get()));
+
+  @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  protected static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HADOOP.catalogName(),
+        SparkCatalogConfig.HADOOP.implementation(),
+        SparkCatalogConfig.HADOOP.properties()
+      },
+    };
+  }
+
+  @BeforeAll
+  public static void startMetastoreAndSpark() {
+    // First call parent to initialize metastore and spark with local[2]
+    CatalogTestBase.startMetastoreAndSpark();
+
+    // Now stop and recreate spark with local[1] to write all rows to a single 
file
+    if (spark != null) {
+      spark.stop();
+    }
+
+    spark =
+        SparkSession.builder()
+            .master("local[1]") // Use one thread to write the rows to a 
single parquet file
+            .config("spark.driver.host", 
InetAddress.getLoopbackAddress().getHostAddress())
+            .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic")
+            .config("spark.hadoop." + METASTOREURIS.varname, 
hiveConf.get(METASTOREURIS.varname))
+            
.config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true")
+            .enableHiveSupport()
+            .getOrCreate();
+
+    sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+  }
+
+  @BeforeEach
+  public void before() {
+    super.before();
+    validationCatalog.createTable(
+        tableIdent, SCHEMA, null, Map.of(TableProperties.FORMAT_VERSION, "3"));
+  }
+
+  @AfterEach
+  public void after() {
+    spark.conf().unset(SparkSQLProperties.SHRED_VARIANTS);
+    spark.conf().unset(SparkSQLProperties.VARIANT_INFERENCE_BUFFER_SIZE);
+    validationCatalog.dropTable(tableIdent, true);
+  }
+
+  @TestTemplate
+  public void testVariantShreddingDisabled() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "false");
+
+    String values = "(1, parse_json('{\"city\": \"NYC\", \"zip\": 10001}')), 
(2, null)";
+    sql("INSERT INTO %s VALUES %s", tableName, values);
+
+    GroupType address = variant("address", 2, Type.Repetition.OPTIONAL);
+    MessageType expectedSchema = parquetSchema(address);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    verifyParquetSchema(table, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testExcludingNullValue() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "true");
+
+    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", tableName, 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);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    verifyParquetSchema(table, expectedSchema);
+  }
+
+  @TestTemplate
+  public void testInconsistentType() throws IOException {
+    spark.conf().set(SparkSQLProperties.SHRED_VARIANTS, "true");
+
+    String values =
+        "(1, parse_json('{\"age\": \"25\"}')),"
+            + " (2, parse_json('{\"age\": 30}')),"
+            + " (3, parse_json('{\"age\": \"35\"}'))";
+    sql("INSERT INTO %s VALUES %s", tableName, 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);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    verifyParquetSchema(table, expectedSchema);

Review Comment:
   I'd add a test to make sure that row 2 had age of value 30:int, just to make 
sure that the parser hasn't decided to "be helpful" 



##########
core/src/main/java/org/apache/iceberg/io/BufferedFileAppender.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.io;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * A FileAppender that buffers the first N rows, then creates a delegate 
appender via a factory.
+ *
+ * <p>The factory receives the buffered rows, is responsible for creating the 
real appender and
+ * writing the buffered rows into it before returning. All subsequent {@link 
#add} calls delegate
+ * directly to the real appender.
+ *
+ * <p>If fewer than N rows are written before {@link #close}, the factory is 
called at close time.
+ *
+ * @param <D> the row type
+ */
+public class BufferedFileAppender<D> implements FileAppender<D> {
+  private final int bufferRowCount;
+  private final Function<List<D>, FileAppender<D>> appenderFactory;
+  private final UnaryOperator<D> copyFunc;
+  private List<D> buffer;
+  private FileAppender<D> delegate;
+  private boolean closed = false;
+
+  /**
+   * @param bufferRowCount number of rows to buffer before creating the 
delegate appender
+   * @param appenderFactory given the buffered rows, creates the delegate 
appender and replays them
+   * @param copyFunc copies a row before buffering (needed when row objects 
are reused, e.g. Spark
+   *     InternalRow)
+   */
+  public BufferedFileAppender(
+      int bufferRowCount,
+      Function<List<D>, FileAppender<D>> appenderFactory,
+      UnaryOperator<D> copyFunc) {
+    Preconditions.checkArgument(
+        bufferRowCount > 0, "bufferRowCount must be > 0, got %s", 
bufferRowCount);
+    Preconditions.checkNotNull(appenderFactory, "appenderFactory must not be 
null");
+    Preconditions.checkNotNull(copyFunc, "copyFunc must not be null");
+    this.bufferRowCount = bufferRowCount;
+    this.appenderFactory = appenderFactory;
+    this.copyFunc = copyFunc;
+    this.buffer = Lists.newArrayList();
+  }
+
+  @Override
+  public void add(D datum) {
+    Preconditions.checkState(!closed, "Cannot add to a closed appender");
+    if (delegate != null) {
+      delegate.add(datum);
+    } else {
+      buffer.add(copyFunc.apply(datum));
+      if (buffer.size() >= bufferRowCount) {
+        initialize();
+      }
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    Preconditions.checkState(closed, "Cannot return metrics for unclosed 
appender");
+    Preconditions.checkState(delegate != null, "Delegate appender was never 
created");
+    return delegate.metrics();
+  }
+
+  @Override
+  public long length() {
+    if (delegate != null) {
+      return delegate.length();
+    }
+    return 0L;
+  }
+
+  @Override
+  public List<Long> splitOffsets() {
+    if (delegate != null) {
+      return delegate.splitOffsets();
+    }
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {

Review Comment:
   what happens on a create().close() sequence with no data written? it should 
be a no-op. Is this tested?



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