the-other-tim-brown commented on code in PR #17833:
URL: https://github.com/apache/hudi/pull/17833#discussion_r2710352152


##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestVariantCrossEngineCompatibility.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.hudi.table;
+
+import org.apache.hudi.common.testutils.ZipTestUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.utils.FlinkMiniCluster;
+import org.apache.hudi.utils.TestTableEnvs;
+
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+/**
+ * Integration test for cross-engine compatibility - verifying that Flink can 
read Variant tables written by Spark 4.0.
+ */
+@ExtendWith(FlinkMiniCluster.class)
+public class ITTestVariantCrossEngineCompatibility {
+
+  @TempDir
+  Path tempDir;
+
+  /**
+   * Helper method to verify that Flink can read Spark 4.0 Variant tables.
+   * Variant data is represented as ROW<value BYTES, metadata BYTES> in Flink.
+   */
+  private void verifyFlinkCanReadSparkVariantTable(String tablePath, String 
tableType, String testDescription) throws Exception {
+    TableEnvironment tableEnv = TestTableEnvs.getBatchTableEnv();
+
+    // Create a Hudi table pointing to the Spark-written data
+    // In Flink, Variant is represented as ROW<value BYTES, metadata BYTES>
+    // NOTE: value is a reserved keyword
+    String createTableDdl = String.format(
+        "CREATE TABLE variant_table ("
+            + "  id INT,"
+            + "  name STRING,"
+            + "  v ROW<`value` BYTES, metadata BYTES>,"

Review Comment:
   Is there a way to create the table with the HoodieSchema so the type is 
annotated as Variant?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala:
##########
@@ -176,4 +179,38 @@ abstract class BaseSpark3Adapter extends SparkAdapter with 
Logging {
     Dataset.ofRows(sqlContext.sparkSession, 
applyFiltersToPlan(logicalRelation, requiredSchema, resolvedSchema,
         
relation.fileFormat.asInstanceOf[HoodieFormatTrait].getRequiredFilters))
   }
+
+  override def getVariantDataType: Option[DataType] = {
+    // Spark 3.x does not support VariantType
+    None
+  }
+
+  override def isDataTypeEqualForParquet(requiredType: DataType, fileType: 
DataType): Option[Boolean] = {
+    // Spark 3.x does not support VariantType, so return None to use default 
logic
+    None
+  }
+
+  override def isVariantType(dataType: DataType): Boolean = {
+    // Spark 3.x does not support VariantType
+    false
+  }
+
+  override def createVariantValueWriter(
+    dataType: DataType,
+    writeValue: Consumer[Array[Byte]],
+    writeMetadata: Consumer[Array[Byte]]
+  ): BiConsumer[SpecializedGetters, Integer] = {
+    // Spark 3.x does support VariantType
+    null
+  }
+
+  override def convertVariantFieldToParquetType(
+    dataType: DataType,
+    fieldName: String,
+    fieldSchema: HoodieSchema,
+    repetition: Repetition
+  ): Type = {
+    // Spark 3.x does not support VariantType
+    null

Review Comment:
   Same here



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java:
##########
@@ -280,6 +281,23 @@ private ValueWriter makeWriter(HoodieSchema schema, 
DataType dataType) {
     } else if (dataType == DataTypes.BinaryType) {
       return (row, ordinal) -> recordConsumer.addBinary(
           Binary.fromReusedByteArray(row.getBinary(ordinal)));
+    } else if 
(SparkAdapterSupport$.MODULE$.sparkAdapter().isVariantType(dataType)) {

Review Comment:
   Do we need something similar in `HoodieAvroWriteSupport`?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSparkVariant.scala:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.hudi.common.table.read
+
+import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.testutils.{HoodieTestTable, HoodieTestUtils}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.storage.StorageConfiguration
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration
+import org.apache.hudi.util.CloseableInternalRowIterator
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.{HoodieSparkKryoRegistrar, SparkConf}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import 
org.apache.spark.sql.internal.SQLConf.LEGACY_RESPECT_NULLABILITY_IN_TEXT_DATASET_CONVERSION
+import org.apache.spark.sql.types.StructType
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.api.Assertions.{assertArrayEquals, assertEquals}
+import org.junit.jupiter.api.Assumptions.assumeTrue
+
+import java.lang.Exception
+import java.nio.file.{Files, Path}
+
+class TestHoodieFileGroupReaderOnSparkVariant extends SparkAdapterSupport {
+  var spark: SparkSession = _
+  var tempDir: Path = _
+
+  @BeforeEach
+  def setup(): Unit = {
+    val sparkConf = new SparkConf
+    sparkConf.set("spark.app.name", getClass.getName)
+    sparkConf.set("spark.master", "local[8]")
+    sparkConf.set("spark.default.parallelism", "4")
+    sparkConf.set("spark.sql.shuffle.partitions", "4")
+    sparkConf.set("spark.driver.maxResultSize", "2g")
+    sparkConf.set("spark.serializer", 
"org.apache.spark.serializer.KryoSerializer")
+    sparkConf.set("spark.kryo.registrator", 
"org.apache.spark.HoodieSparkKryoRegistrar")
+    sparkConf.set("spark.sql.extensions", 
"org.apache.spark.sql.hudi.HoodieSparkSessionExtension")
+    sparkConf.set("spark.sql.parquet.enableVectorizedReader", "false")
+    sparkConf.set("spark.sql.orc.enableVectorizedReader", "false")
+    sparkConf.set(LEGACY_RESPECT_NULLABILITY_IN_TEXT_DATASET_CONVERSION.key, 
"true")
+    HoodieSparkKryoRegistrar.register(sparkConf)
+    spark = SparkSession.builder.config(sparkConf).getOrCreate
+    tempDir = Files.createTempDirectory("test_variant_")
+  }
+
+  @AfterEach
+  def teardown(): Unit = {
+    if (spark != null) {
+      spark.stop()
+    }
+  }
+
+  def getStorageConf: StorageConfiguration[_] = {
+    HoodieTestUtils.getDefaultStorageConf.getInline
+  }
+
+  def getBasePath: String = {
+    tempDir.toAbsolutePath.toUri.toString
+  }
+
+  @Test
+  def testReadVariantDataType(): Unit = {
+    // Variant type is only supported in Spark 4.0+
+    assumeTrue(HoodieSparkUtils.gteqSpark4_0, "Variant type requires Spark 4.0 
or higher")
+
+    val tableName = "test_variant_table"
+
+    // Create table with variant column
+    spark.sql(
+      s"""
+         |create table $tableName (
+         |  id int,
+         |  name string,
+         |  v variant,
+         |  b binary,
+         |  ts long
+         |) using hudi
+         | location '$getBasePath'
+         | tblproperties (
+         |  primaryKey = 'id',
+         |  type = 'mor',
+         |  preCombineField = 'ts'
+         | )
+       """.stripMargin)
+
+    // Insert variant data
+    spark.sql(
+      s"""
+         |insert into $tableName
+         |values
+         |  (1, 'row1', parse_json('{"key": "value1", "num": 1}'), 
X'0102030405', 1000),
+         |  (2, 'row2', parse_json('{"key": "value2", "list": [1, 2, 3]}'), 
X'0504030201', 1000)
+       """.stripMargin)
+
+    // Update variant data
+    spark.sql(
+      s"""
+         |update $tableName
+         |set v = parse_json('{"updated": true, "new_field": 123}')
+         |where id = 1
+       """.stripMargin)
+
+    // Get metaClient and base files
+    val metaClient = HoodieTableMetaClient.builder()
+      .setConf(getStorageConf)
+      .setBasePath(getBasePath)
+      .build()
+    val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles
+    assertEquals(1, allBaseFiles.size())
+
+    // Create parquet reader
+    val hadoopConf = new Configuration(spark.sparkContext.hadoopConfiguration)
+    val reader = sparkAdapter.createParquetFileReader(

Review Comment:
   Instead of using a parquetFileReader directly, can we use a FileGroupReader? 
I'm thinking later it will be useful when MoR is incorporated.



##########
hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark4Adapter.scala:
##########
@@ -196,4 +198,88 @@ abstract class BaseSpark4Adapter extends SparkAdapter with 
Logging {
       storageConf.getBoolean(SQLConf.CASE_SENSITIVE.key, 
sqlConf.caseSensitiveAnalysis),
       getRebaseSpec("CORRECTED"))
   }
+
+  override def getVariantDataType: Option[DataType] = {
+    Some(VariantType)
+  }
+
+  override def isDataTypeEqualForParquet(requiredType: DataType, fileType: 
DataType): Option[Boolean] = {
+    import org.apache.spark.sql.types.{BinaryType, StructType, VariantType}
+
+    /**
+     * Checks if a StructType is the physical representation of VariantType in 
Parquet.
+     * VariantType is stored in Parquet as a struct with two binary fields: 
"value" and "metadata".
+     */
+    def isVariantPhysicalSchema(structType: StructType): Boolean = {
+      if (structType.fields.length != 2) {
+        false
+      } else {
+        val fieldMap = structType.fields.map(f => (f.name, f.dataType)).toMap
+        fieldMap.contains("value") && fieldMap.contains("metadata") &&
+          fieldMap("value") == BinaryType && fieldMap("metadata") == BinaryType
+      }
+    }
+
+    // Handle VariantType comparisons
+    (requiredType, fileType) match {
+      case (_: VariantType, s: StructType) if isVariantPhysicalSchema(s) => 
Some(true)

Review Comment:
   Why wouldn't the file's type also be Variant?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSparkVariant.scala:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.hudi.common.table.read
+
+import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport}
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.testutils.{HoodieTestTable, HoodieTestUtils}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.storage.StorageConfiguration
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration
+import org.apache.hudi.util.CloseableInternalRowIterator
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.{HoodieSparkKryoRegistrar, SparkConf}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import 
org.apache.spark.sql.internal.SQLConf.LEGACY_RESPECT_NULLABILITY_IN_TEXT_DATASET_CONVERSION
+import org.apache.spark.sql.types.StructType
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.api.Assertions.{assertArrayEquals, assertEquals}
+import org.junit.jupiter.api.Assumptions.assumeTrue
+
+import java.lang.Exception
+import java.nio.file.{Files, Path}
+
+class TestHoodieFileGroupReaderOnSparkVariant extends SparkAdapterSupport {
+  var spark: SparkSession = _
+  var tempDir: Path = _
+
+  @BeforeEach
+  def setup(): Unit = {
+    val sparkConf = new SparkConf
+    sparkConf.set("spark.app.name", getClass.getName)
+    sparkConf.set("spark.master", "local[8]")
+    sparkConf.set("spark.default.parallelism", "4")
+    sparkConf.set("spark.sql.shuffle.partitions", "4")
+    sparkConf.set("spark.driver.maxResultSize", "2g")
+    sparkConf.set("spark.serializer", 
"org.apache.spark.serializer.KryoSerializer")
+    sparkConf.set("spark.kryo.registrator", 
"org.apache.spark.HoodieSparkKryoRegistrar")
+    sparkConf.set("spark.sql.extensions", 
"org.apache.spark.sql.hudi.HoodieSparkSessionExtension")
+    sparkConf.set("spark.sql.parquet.enableVectorizedReader", "false")
+    sparkConf.set("spark.sql.orc.enableVectorizedReader", "false")
+    sparkConf.set(LEGACY_RESPECT_NULLABILITY_IN_TEXT_DATASET_CONVERSION.key, 
"true")
+    HoodieSparkKryoRegistrar.register(sparkConf)
+    spark = SparkSession.builder.config(sparkConf).getOrCreate
+    tempDir = Files.createTempDirectory("test_variant_")
+  }
+
+  @AfterEach
+  def teardown(): Unit = {
+    if (spark != null) {
+      spark.stop()
+    }
+  }
+
+  def getStorageConf: StorageConfiguration[_] = {
+    HoodieTestUtils.getDefaultStorageConf.getInline
+  }
+
+  def getBasePath: String = {
+    tempDir.toAbsolutePath.toUri.toString
+  }
+
+  @Test
+  def testReadVariantDataType(): Unit = {
+    // Variant type is only supported in Spark 4.0+
+    assumeTrue(HoodieSparkUtils.gteqSpark4_0, "Variant type requires Spark 4.0 
or higher")
+
+    val tableName = "test_variant_table"
+
+    // Create table with variant column
+    spark.sql(
+      s"""
+         |create table $tableName (
+         |  id int,
+         |  name string,
+         |  v variant,
+         |  b binary,
+         |  ts long
+         |) using hudi
+         | location '$getBasePath'
+         | tblproperties (
+         |  primaryKey = 'id',
+         |  type = 'mor',
+         |  preCombineField = 'ts'
+         | )
+       """.stripMargin)
+
+    // Insert variant data
+    spark.sql(
+      s"""
+         |insert into $tableName
+         |values
+         |  (1, 'row1', parse_json('{"key": "value1", "num": 1}'), 
X'0102030405', 1000),
+         |  (2, 'row2', parse_json('{"key": "value2", "list": [1, 2, 3]}'), 
X'0504030201', 1000)
+       """.stripMargin)
+
+    // Update variant data
+    spark.sql(
+      s"""
+         |update $tableName
+         |set v = parse_json('{"updated": true, "new_field": 123}')
+         |where id = 1
+       """.stripMargin)
+
+    // Get metaClient and base files
+    val metaClient = HoodieTableMetaClient.builder()
+      .setConf(getStorageConf)
+      .setBasePath(getBasePath)
+      .build()
+    val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles
+    assertEquals(1, allBaseFiles.size())
+
+    // Create parquet reader
+    val hadoopConf = new Configuration(spark.sparkContext.hadoopConfiguration)
+    val reader = sparkAdapter.createParquetFileReader(
+      vectorized = false,
+      spark.sessionState.conf,
+      Map.empty,
+      hadoopConf)
+
+    // Get the schema for the table
+    val dataSchema = spark.table(tableName).schema
+
+    // Create partitioned file for reading
+    val fileInfo = 
sparkAdapter.getSparkPartitionedFileUtils.createPartitionedFile(
+      InternalRow.empty,
+      allBaseFiles.get(0).getPath,
+      0,
+      allBaseFiles.get(0).getLength)
+
+    // Read using CloseableInternalRowIterator
+    val iterator = new CloseableInternalRowIterator(
+      reader.read(
+        fileInfo,
+        dataSchema,
+        StructType(Seq.empty),
+        HOption.empty(),
+        Seq.empty,
+        new HadoopStorageConfiguration(hadoopConf)))
+
+    // Validate we can read variant data for id = 1
+    while (iterator.hasNext) {
+      val row = iterator.next()
+      val id = row.getInt(5)
+
+      if (id == 1) {
+        val name = row.getUTF8String(6).toString

Review Comment:
   Instead of doing individual field inspection, is it possible to compare rows 
directly with an expected set of rows?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala:
##########
@@ -176,4 +179,38 @@ abstract class BaseSpark3Adapter extends SparkAdapter with 
Logging {
     Dataset.ofRows(sqlContext.sparkSession, 
applyFiltersToPlan(logicalRelation, requiredSchema, resolvedSchema,
         
relation.fileFormat.asInstanceOf[HoodieFormatTrait].getRequiredFilters))
   }
+
+  override def getVariantDataType: Option[DataType] = {
+    // Spark 3.x does not support VariantType
+    None
+  }
+
+  override def isDataTypeEqualForParquet(requiredType: DataType, fileType: 
DataType): Option[Boolean] = {
+    // Spark 3.x does not support VariantType, so return None to use default 
logic
+    None
+  }
+
+  override def isVariantType(dataType: DataType): Boolean = {
+    // Spark 3.x does not support VariantType
+    false
+  }
+
+  override def createVariantValueWriter(
+    dataType: DataType,
+    writeValue: Consumer[Array[Byte]],
+    writeMetadata: Consumer[Array[Byte]]
+  ): BiConsumer[SpecializedGetters, Integer] = {
+    // Spark 3.x does support VariantType
+    null

Review Comment:
   Let's have this throw an `UnsupportedOperationException` 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to