This is an automated email from the ASF dual-hosted git repository.

yuanzhou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git


The following commit(s) were added to refs/heads/main by this push:
     new a3f86ede1 [VL] Add timetravel and partition filter UTs for Iceberg 
scan (#5092)
a3f86ede1 is described below

commit a3f86ede1c5f9ab44353d124ab57be5bc1a0561e
Author: Yan Ma <[email protected]>
AuthorDate: Tue Mar 26 09:23:49 2024 +0800

    [VL] Add timetravel and partition filter UTs for Iceberg scan (#5092)
---
 .../execution/VeloxIcebergSuite.scala              | 38 ++++++++++++++++++++++
 1 file changed, 38 insertions(+)

diff --git 
a/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
 
b/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
index 6b332641e..9dc933562 100644
--- 
a/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
+++ 
b/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
@@ -19,6 +19,7 @@ package io.glutenproject.execution
 import io.glutenproject.GlutenConfig
 
 import org.apache.spark.SparkConf
+import org.apache.spark.sql.Row
 
 class VeloxIcebergSuite extends WholeStageTransformerSuite {
 
@@ -317,6 +318,43 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite 
{
     }
   }
 
+  test("iceberg: time travel") {
+    withTable("iceberg_tm") {
+      spark.sql(s"""
+                   |create table iceberg_tm (id int, name string) using iceberg
+                   |""".stripMargin)
+      spark.sql(s"""
+                   |insert into iceberg_tm values (1, "v1"), (2, "v2")
+                   |""".stripMargin)
+      spark.sql(s"""
+                   |insert into iceberg_tm values (3, "v3"), (4, "v4")
+                   |""".stripMargin)
+
+      val df =
+        spark.sql("select snapshot_id from default.iceberg_tm.snapshots where 
parent_id is null")
+      val value = df.collectAsList().get(0).getAs[Long](0);
+      spark.sql(s"call 
system.set_current_snapshot('default.iceberg_tm',$value)");
+      val data = runQueryAndCompare("select * from iceberg_tm") { _ => }
+      checkLengthAndPlan(data, 2)
+      checkAnswer(data, Row(1, "v1") :: Row(2, "v2") :: Nil)
+    }
+  }
+
+  test("iceberg: partition filters") {
+    withTable("iceberg_pf") {
+      spark.sql(s"""
+                   |create table iceberg_pf (id int, name string)
+                   | using iceberg partitioned by (name)
+                   |""".stripMargin)
+      spark.sql(s"""
+                   |insert into iceberg_pf values (1, "v1"), (2, "v2"), (3, 
"v1"), (4, "v2")
+                   |""".stripMargin)
+      val df1 = runQueryAndCompare("select * from iceberg_pf where name = 
'v1'") { _ => }
+      checkLengthAndPlan(df1, 2)
+      checkAnswer(df1, Row(1, "v1") :: Row(3, "v1") :: Nil)
+    }
+  }
+
   test("iceberg read mor table - delete and update") {
     withTable("iceberg_mor_tb") {
       withSQLConf(GlutenConfig.GLUTEN_ENABLE_KEY -> "false") {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to