voonhous commented on code in PR #17833:
URL: https://github.com/apache/hudi/pull/17833#discussion_r2735400606


##########
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:
   Yes. It will have a variant logical type in HoodieSchema.



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