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

philo 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 adf0566a2 [VL][MINOR] Refactor operator/function validation tests 
(#5037)
adf0566a2 is described below

commit adf0566a2056276612694bc980f4e6e9028eb7d1
Author: PHILO-HE <feilong...@intel.com>
AuthorDate: Tue Mar 26 20:12:28 2024 +0800

    [VL][MINOR] Refactor operator/function validation tests (#5037)
---
 ...lutenClickHouseWholeStageTransformerSuite.scala |   1 -
 .../benchmarks/NativeBenchmarkPlanGenerator.scala  |   1 -
 .../benchmarks/ShuffleWriterFuzzerTest.scala       |   1 -
 .../io/glutenproject/execution/FallbackSuite.scala |   1 -
 ...sionSuite.scala => FunctionsValidateTest.scala} |  71 ++++-----
 ...te.scala => ScalarFunctionsValidateSuite.scala} | 162 +++++++++++----------
 .../io/glutenproject/execution/TestOperator.scala  | 135 ++---------------
 .../execution/VeloxAggregateFunctionsSuite.scala   |   1 -
 .../execution/VeloxColumnarCacheSuite.scala        |   1 -
 .../execution/VeloxHashJoinSuite.scala             |   1 -
 .../execution/VeloxLiteralSuite.scala              |   1 -
 .../execution/VeloxMetricsSuite.scala              |   1 -
 .../VeloxOrcDataTypeValidationSuite.scala          |   1 -
 .../VeloxParquetDataTypeValidationSuite.scala      |   1 -
 .../glutenproject/execution/VeloxScanSuite.scala   |   1 -
 .../execution/VeloxStringFunctionsSuite.scala      |   1 -
 .../glutenproject/execution/VeloxTPCDSSuite.scala  |   1 -
 .../glutenproject/execution/VeloxTPCHSuite.scala   |   1 -
 .../execution/VeloxWindowExpressionSuite.scala     |   1 -
 .../execution/WindowFunctionsValidateSuite.scala   |  35 +++++
 .../sql/execution/VeloxParquetWriteSuite.scala     |   1 -
 .../execution/WholeStageTransformerSuite.scala     |   1 -
 .../glutenproject/execution/VeloxDeltaSuite.scala  |   1 -
 .../execution/VeloxIcebergSuite.scala              |   1 -
 24 files changed, 175 insertions(+), 248 deletions(-)

diff --git 
a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseWholeStageTransformerSuite.scala
 
b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseWholeStageTransformerSuite.scala
index e40f3d0e7..a2de7cf51 100644
--- 
a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseWholeStageTransformerSuite.scala
+++ 
b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseWholeStageTransformerSuite.scala
@@ -70,7 +70,6 @@ class GlutenClickHouseWholeStageTransformerSuite extends 
WholeStageTransformerSu
   protected val metaStorePathAbsolute = basePath + "/meta"
   protected val hiveMetaStoreDB = metaStorePathAbsolute + "/metastore_db"
 
-  override protected val backend: String = "ch"
   final override protected val resourcePath: String = "" // ch not need this
   override protected val fileFormat: String = "parquet"
 }
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala
 
b/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala
index c9863111a..dafe3af3e 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala
@@ -35,7 +35,6 @@ import scala.collection.JavaConverters._
 object GenerateExample extends Tag("io.glutenproject.tags.GenerateExample")
 
 class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
   val generatedPlanDir = getClass.getResource("/").getPath + 
"../../../generated-native-benchmark/"
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/benchmarks/ShuffleWriterFuzzerTest.scala
 
b/backends-velox/src/test/scala/io/glutenproject/benchmarks/ShuffleWriterFuzzerTest.scala
index 9d723f04f..7f863de68 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/benchmarks/ShuffleWriterFuzzerTest.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/benchmarks/ShuffleWriterFuzzerTest.scala
@@ -37,7 +37,6 @@ object ShuffleWriterFuzzerTest {
 @FuzzerTest
 @SkipTestTags
 class ShuffleWriterFuzzerTest extends VeloxWholeStageTransformerSuite {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala 
b/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala
index d9b1b4604..69e5b614c 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala
@@ -25,7 +25,6 @@ import 
org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, AQEShuf
 
 class FallbackSuite extends VeloxWholeStageTransformerSuite with 
AdaptiveSparkPlanHelper {
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/FunctionsValidateTest.scala
similarity index 56%
copy from 
backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
copy to 
backends-velox/src/test/scala/io/glutenproject/execution/FunctionsValidateTest.scala
index cd93923b0..c91ca8866 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/FunctionsValidateTest.scala
@@ -17,18 +17,18 @@
 package io.glutenproject.execution
 
 import org.apache.spark.SparkConf
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, 
NullPropagation}
+import org.apache.spark.sql.types._
 
-class VeloxWindowExpressionSuite extends WholeStageTransformerSuite {
+import java.nio.file.Files
 
-  protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
+import scala.collection.JavaConverters._
+
+class FunctionsValidateTest extends WholeStageTransformerSuite {
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-    createTPCHNotNullTables()
-  }
+  private var parquetPath: String = _
 
   override protected def sparkConf: SparkConf = {
     super.sparkConf
@@ -39,35 +39,40 @@ class VeloxWindowExpressionSuite extends 
WholeStageTransformerSuite {
       .set("spark.unsafe.exceptionOnMemoryLeak", "true")
       .set("spark.sql.autoBroadcastJoinThreshold", "-1")
       .set("spark.sql.sources.useV1SourceList", "avro")
+      .set(
+        "spark.sql.optimizer.excludedRules",
+        ConstantFolding.ruleName + "," +
+          NullPropagation.ruleName)
   }
 
-  test("window row frame with mix preceding and following") {
-    runQueryAndCompare(
-      "select max(l_suppkey) over" +
-        " (partition by l_suppkey order by l_orderkey " +
-        "rows between 2 preceding and 1 preceding) from lineitem ") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    createTPCHNotNullTables()
+
+    val lfile = Files.createTempFile("", ".parquet").toFile
+    lfile.deleteOnExit()
+    parquetPath = lfile.getAbsolutePath
 
-    runQueryAndCompare(
-      "select max(l_suppkey) over" +
-        " (partition by l_suppkey order by l_orderkey " +
-        "rows between 2 following and 3 following) from lineitem ") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
+    val schema = StructType(
+      Array(
+        StructField("double_field1", DoubleType, true),
+        StructField("int_field1", IntegerType, true),
+        StructField("string_field1", StringType, true)
+      ))
+    val rowData = Seq(
+      Row(1.025, 1, "{\"a\":\"b\"}"),
+      Row(1.035, 2, null),
+      Row(1.045, 3, null)
+    )
 
-    runQueryAndCompare(
-      "select max(l_suppkey) over" +
-        " (partition by l_suppkey order by l_orderkey " +
-        "rows between -3 following and -2 following) from lineitem ") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
+    val dfParquet = spark.createDataFrame(rowData.asJava, schema)
+    dfParquet
+      .coalesce(1)
+      .write
+      .format("parquet")
+      .mode("overwrite")
+      .parquet(parquetPath)
 
-    runQueryAndCompare(
-      "select max(l_suppkey) over" +
-        " (partition by l_suppkey order by l_orderkey " +
-        "rows between unbounded preceding and 3 following) from lineitem ") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
+    spark.catalog.createTable("datatab", parquetPath, fileFormat)
   }
 }
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxFunctionsValidateSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/ScalarFunctionsValidateSuite.scala
similarity index 85%
rename from 
backends-velox/src/test/scala/io/glutenproject/execution/VeloxFunctionsValidateSuite.scala
rename to 
backends-velox/src/test/scala/io/glutenproject/execution/ScalarFunctionsValidateSuite.scala
index ef9c80c4e..f6998b69e 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxFunctionsValidateSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/ScalarFunctionsValidateSuite.scala
@@ -16,69 +16,15 @@
  */
 package io.glutenproject.execution
 
-import org.apache.spark.SparkConf
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, 
NullPropagation}
 import org.apache.spark.sql.types._
 
-import java.nio.file.Files
-
-import scala.collection.JavaConverters._
-
-class VeloxFunctionsValidateSuite extends VeloxWholeStageTransformerSuite {
-
-  override protected val resourcePath: String = "/tpch-data-parquet-velox"
-  override protected val fileFormat: String = "parquet"
-  override protected val backend: String = "velox"
-
-  private var parquetPath: String = _
-
+class ScalarFunctionsValidateSuite extends FunctionsValidateTest {
+  disableFallbackCheck
   import testImplicits._
 
-  override protected def sparkConf: SparkConf = {
-    super.sparkConf
-      .set("spark.shuffle.manager", 
"org.apache.spark.shuffle.sort.ColumnarShuffleManager")
-      .set("spark.sql.files.maxPartitionBytes", "1g")
-      .set("spark.sql.shuffle.partitions", "1")
-      .set("spark.memory.offHeap.size", "2g")
-      .set("spark.unsafe.exceptionOnMemoryLeak", "true")
-      .set("spark.sql.autoBroadcastJoinThreshold", "-1")
-      .set("spark.sql.sources.useV1SourceList", "avro")
-      .set(
-        "spark.sql.optimizer.excludedRules",
-        ConstantFolding.ruleName + "," +
-          NullPropagation.ruleName)
-  }
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-    createTPCHNotNullTables()
-
-    val lfile = Files.createTempFile("", ".parquet").toFile
-    lfile.deleteOnExit()
-    parquetPath = lfile.getAbsolutePath
-
-    val schema = StructType(
-      Array(
-        StructField("double_field1", DoubleType, true),
-        StructField("int_field1", IntegerType, true),
-        StructField("string_field1", StringType, true)
-      ))
-    val rowData = Seq(
-      Row(1.025, 1, "{\"a\":\"b\"}"),
-      Row(1.035, 2, null),
-      Row(1.045, 3, null)
-    )
-
-    var dfParquet = spark.createDataFrame(rowData.asJava, schema)
-    dfParquet
-      .coalesce(1)
-      .write
-      .format("parquet")
-      .mode("overwrite")
-      .parquet(parquetPath)
-
-    spark.catalog.createTable("datatab", parquetPath, fileFormat)
+  // Test "SELECT ..." without a from clause.
+  test("isnull function") {
+    runQueryAndCompare("SELECT 
isnull(1)")(checkOperatorMatch[ProjectExecTransformer])
   }
 
   test("Test bit_count function") {
@@ -135,12 +81,86 @@ class VeloxFunctionsValidateSuite extends 
VeloxWholeStageTransformerSuite {
     }
   }
 
-  ignore("Test round function") {
+  test("Test round function") {
     runQueryAndCompare(
       "SELECT round(cast(l_orderkey as int), 2)" +
         "from lineitem limit 1") {
       checkOperatorMatch[ProjectExecTransformer]
     }
+
+    runQueryAndCompare("""
+                         |select round(l_quantity, 2) from lineitem;
+                         |""".stripMargin) {
+      checkOperatorMatch[ProjectExecTransformer]
+    }
+  }
+
+  test("chr function") {
+    val df = runQueryAndCompare(
+      "SELECT chr(l_orderkey + 64) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("bin function") {
+    val df = runQueryAndCompare(
+      "SELECT bin(l_orderkey) " +
+        "from lineitem limit 1") {
+      checkOperatorMatch[ProjectExecTransformer]
+    }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("abs function") {
+    val df = runQueryAndCompare(
+      "SELECT abs(l_orderkey) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("ceil function") {
+    val df = runQueryAndCompare(
+      "SELECT ceil(cast(l_orderkey as long)) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("floor function") {
+    val df = runQueryAndCompare(
+      "SELECT floor(cast(l_orderkey as long)) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("exp function") {
+    val df = spark.sql("SELECT exp(l_orderkey) from lineitem limit 1")
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("power function") {
+    val df = runQueryAndCompare(
+      "SELECT power(l_orderkey, 2.0) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("pmod function") {
+    val df = runQueryAndCompare(
+      "SELECT pmod(cast(l_orderkey as int), 3) " +
+        "from lineitem limit 1") { _ => }
+    checkLengthAndPlan(df, 1)
+  }
+
+  test("greatest function") {
+    val df = runQueryAndCompare(
+      "SELECT greatest(l_orderkey, l_orderkey)" +
+        "from lineitem limit 1")(checkOperatorMatch[ProjectExecTransformer])
+  }
+
+  test("least function") {
+    val df = runQueryAndCompare(
+      "SELECT least(l_orderkey, l_orderkey)" +
+        "from lineitem limit 1")(checkOperatorMatch[ProjectExecTransformer])
   }
 
   test("Test greatest function") {
@@ -344,6 +364,14 @@ class VeloxFunctionsValidateSuite extends 
VeloxWholeStageTransformerSuite {
     }
   }
 
+  test("Support HOUR function") {
+    withTable("t1") {
+      sql("create table t1 (c1 int, c2 timestamp) USING PARQUET")
+      sql("INSERT INTO t1 VALUES(1, NOW())")
+      runQueryAndCompare("SELECT c1, HOUR(c2) FROM t1 LIMIT 1")(df => 
checkFallbackOperators(df, 0))
+    }
+  }
+
   test("map extract - getmapvalue") {
     withTempPath {
       path =>
@@ -545,20 +573,6 @@ class VeloxFunctionsValidateSuite extends 
VeloxWholeStageTransformerSuite {
     }
   }
 
-  test("lag/lead window function with negative input offset") {
-    runQueryAndCompare(
-      "select lag(l_orderkey, -2) over" +
-        " (partition by l_suppkey order by l_orderkey) from lineitem") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
-
-    runQueryAndCompare(
-      "select lead(l_orderkey, -2) over" +
-        " (partition by l_suppkey order by l_orderkey) from lineitem") {
-      checkOperatorMatch[WindowExecTransformer]
-    }
-  }
-
   test("bit_length") {
     runQueryAndCompare(
       "select bit_length(c_comment), bit_length(cast(c_comment as binary))" +
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala 
b/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala
index 55f8ee6f5..be3a50f1e 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala
@@ -20,7 +20,7 @@ import io.glutenproject.GlutenConfig
 import io.glutenproject.sql.shims.SparkShimLoader
 
 import org.apache.spark.SparkConf
-import org.apache.spark.sql.{AnalysisException, DataFrame, Row}
+import org.apache.spark.sql.{AnalysisException, Row}
 import org.apache.spark.sql.execution.{FilterExec, GenerateExec, ProjectExec, 
RDDScanExec}
 import org.apache.spark.sql.functions.{avg, col, lit, udf}
 import org.apache.spark.sql.internal.SQLConf
@@ -31,7 +31,6 @@ import scala.collection.JavaConverters
 class TestOperator extends VeloxWholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
@@ -197,36 +196,25 @@ class TestOperator extends 
VeloxWholeStageTransformerSuite {
   }
 
   test("window expression") {
-    def assertWindowOffloaded: DataFrame => Unit = {
-      df =>
-        {
-          assert(
-            getExecutedPlan(df).count(
-              plan => {
-                plan.isInstanceOf[WindowExecTransformer]
-              }) > 0)
-        }
-    }
-
     Seq("sort", "streaming").foreach {
       windowType =>
         withSQLConf("spark.gluten.sql.columnar.backend.velox.window.type" -> 
windowType) {
           runQueryAndCompare(
             "select ntile(4) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select row_number() over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select rank() over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
@@ -244,147 +232,68 @@ class TestOperator extends 
VeloxWholeStageTransformerSuite {
           runQueryAndCompare(
             "select l_suppkey, l_orderkey, nth_value(l_orderkey, 2) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select l_suppkey, l_orderkey, nth_value(l_orderkey, 2) IGNORE 
NULLS over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select sum(l_partkey + 1) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select max(l_partkey) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select min(l_partkey) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select avg(l_partkey) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select lag(l_orderkey) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           runQueryAndCompare(
             "select lead(l_orderkey) over" +
               " (partition by l_suppkey order by l_orderkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           // Test same partition/ordering keys.
           runQueryAndCompare(
             "select avg(l_partkey) over" +
               " (partition by l_suppkey order by l_suppkey) from lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
 
           // Test overlapping partition/ordering keys.
           runQueryAndCompare(
             "select avg(l_partkey) over" +
               " (partition by l_suppkey order by l_suppkey, l_orderkey) from 
lineitem ") {
-            assertWindowOffloaded
+            checkOperatorMatch[WindowExecTransformer]
           }
         }
     }
   }
 
-  test("chr function") {
-    val df = runQueryAndCompare(
-      "SELECT chr(l_orderkey + 64) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("bin function") {
-    val df = runQueryAndCompare(
-      "SELECT bin(l_orderkey) " +
-        "from lineitem limit 1") {
-      checkOperatorMatch[ProjectExecTransformer]
-    }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("abs function") {
-    val df = runQueryAndCompare(
-      "SELECT abs(l_orderkey) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("ceil function") {
-    val df = runQueryAndCompare(
-      "SELECT ceil(cast(l_orderkey as long)) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("floor function") {
-    val df = runQueryAndCompare(
-      "SELECT floor(cast(l_orderkey as long)) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("exp function") {
-    val df = spark.sql("SELECT exp(l_orderkey) from lineitem limit 1")
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("power function") {
-    val df = runQueryAndCompare(
-      "SELECT power(l_orderkey, 2.0) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("pmod function") {
-    val df = runQueryAndCompare(
-      "SELECT pmod(cast(l_orderkey as int), 3) " +
-        "from lineitem limit 1") { _ => }
-    checkLengthAndPlan(df, 1)
-  }
-
-  test("round function") {
-    val df = runQueryAndCompare(
-      "SELECT round(cast(l_orderkey as int), 2)" +
-        "from lineitem limit 1")(checkOperatorMatch[ProjectExecTransformer])
-  }
-
-  test("greatest function") {
-    val df = runQueryAndCompare(
-      "SELECT greatest(l_orderkey, l_orderkey)" +
-        "from lineitem limit 1")(checkOperatorMatch[ProjectExecTransformer])
-  }
-
-  test("least function") {
-    val df = runQueryAndCompare(
-      "SELECT least(l_orderkey, l_orderkey)" +
-        "from lineitem limit 1")(checkOperatorMatch[ProjectExecTransformer])
-  }
-
-  // Test "SELECT ..." without a from clause.
-  test("isnull function") {
-    runQueryAndCompare("SELECT isnull(1)") { _ => }
-  }
-
   test("df.count()") {
     val df = runQueryAndCompare("select * from lineitem limit 1") { _ => }
     checkLengthAndPlan(df, 1)
@@ -447,14 +356,6 @@ class TestOperator extends VeloxWholeStageTransformerSuite 
{
     }
   }
 
-  test("round") {
-    runQueryAndCompare("""
-                         |select round(l_quantity, 2) from lineitem;
-                         |""".stripMargin) {
-      checkOperatorMatch[ProjectExecTransformer]
-    }
-  }
-
   test("bool scan") {
     withTempPath {
       path =>
@@ -1133,14 +1034,6 @@ class TestOperator extends 
VeloxWholeStageTransformerSuite {
     }
   }
 
-  test("Support HOUR function") {
-    withTable("t1") {
-      sql("create table t1 (c1 int, c2 timestamp) USING PARQUET")
-      sql("INSERT INTO t1 VALUES(1, NOW())")
-      runQueryAndCompare("SELECT c1, HOUR(c2) FROM t1 LIMIT 1")(df => 
checkFallbackOperators(df, 0))
-    }
-  }
-
   test("Support Array type signature") {
     withTable("t1", "t2") {
       sql("CREATE TABLE t1(id INT, l ARRAY<INT>) USING PARQUET")
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala
index c0143d0ae..c306f4585 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala
@@ -24,7 +24,6 @@ import org.apache.spark.sql.internal.SQLConf
 abstract class VeloxAggregateFunctionsSuite extends 
VeloxWholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala
index 04af5d3ce..b1706d7d8 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala
@@ -27,7 +27,6 @@ import 
org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
 import org.apache.spark.storage.StorageLevel
 
 class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with 
AdaptiveSparkPlanHelper {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxHashJoinSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxHashJoinSuite.scala
index e2cc4dc0b..459220fc7 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxHashJoinSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxHashJoinSuite.scala
@@ -22,7 +22,6 @@ import org.apache.spark.SparkConf
 import org.apache.spark.sql.execution.InputIteratorTransformer
 
 class VeloxHashJoinSuite extends VeloxWholeStageTransformerSuite {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxLiteralSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxLiteralSuite.scala
index 52e122c2b..354c33580 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxLiteralSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxLiteralSuite.scala
@@ -22,7 +22,6 @@ import org.apache.spark.sql.execution.ProjectExec
 class VeloxLiteralSuite extends VeloxWholeStageTransformerSuite {
   override protected val resourcePath: String = "placeholder"
   override protected val fileFormat: String = "parquet"
-  override protected val backend: String = "velox"
 
   override def beforeAll(): Unit = {
     super.beforeAll()
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxMetricsSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxMetricsSuite.scala
index 785a302e5..3110048ee 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxMetricsSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxMetricsSuite.scala
@@ -24,7 +24,6 @@ import 
org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
 import org.apache.spark.sql.internal.SQLConf
 
 class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with 
AdaptiveSparkPlanHelper {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxOrcDataTypeValidationSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxOrcDataTypeValidationSuite.scala
index ff13ce5c4..8e94397c2 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxOrcDataTypeValidationSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxOrcDataTypeValidationSuite.scala
@@ -22,7 +22,6 @@ import java.io.File
 
 class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite {
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/data-type-validation-data"
   override protected val fileFormat: String = "orc"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxParquetDataTypeValidationSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxParquetDataTypeValidationSuite.scala
index 22bb04a6c..c19d35014 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxParquetDataTypeValidationSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxParquetDataTypeValidationSuite.scala
@@ -22,7 +22,6 @@ import java.io.File
 
 class VeloxParquetDataTypeValidationSuite extends 
VeloxWholeStageTransformerSuite {
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/data-type-validation-data"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxScanSuite.scala 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxScanSuite.scala
index 6bf640c94..82aa19b33 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxScanSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxScanSuite.scala
@@ -22,7 +22,6 @@ import org.apache.spark.sql.execution.ScalarSubquery
 
 class VeloxScanSuite extends VeloxWholeStageTransformerSuite {
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala
index ed1f851ca..24bc5aac6 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala
@@ -26,7 +26,6 @@ import org.apache.spark.sql.types.StringType
 class VeloxStringFunctionsSuite extends VeloxWholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCDSSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCDSSuite.scala
index e25b386e7..56f73e40c 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCDSSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCDSSuite.scala
@@ -30,7 +30,6 @@ import scala.io.Source
 // Then set the `ignore` to `test`
 class VeloxTPCDSSuite extends VeloxWholeStageTransformerSuite {
 
-  override protected val backend: String = "velox"
   override protected val resourcePath: String =
     sys.env.getOrElse("SPARK_TPCDS_DATA", "/tmp/tpcds-generated")
   override protected val fileFormat: String = "parquet"
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCHSuite.scala 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCHSuite.scala
index 8bcc16fac..9ac10e56e 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCHSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxTPCHSuite.scala
@@ -27,7 +27,6 @@ import java.nio.charset.StandardCharsets
 
 abstract class VeloxTPCHTableSupport extends VeloxWholeStageTransformerSuite {
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
index cd93923b0..304342a7c 100644
--- 
a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxWindowExpressionSuite.scala
@@ -21,7 +21,6 @@ import org.apache.spark.SparkConf
 class VeloxWindowExpressionSuite extends WholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/backends-velox/src/test/scala/io/glutenproject/execution/WindowFunctionsValidateSuite.scala
 
b/backends-velox/src/test/scala/io/glutenproject/execution/WindowFunctionsValidateSuite.scala
new file mode 100644
index 000000000..9f3defd4f
--- /dev/null
+++ 
b/backends-velox/src/test/scala/io/glutenproject/execution/WindowFunctionsValidateSuite.scala
@@ -0,0 +1,35 @@
+/*
+ * 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 io.glutenproject.execution
+
+class WindowFunctionsValidateSuite extends FunctionsValidateTest {
+
+  test("lag/lead window function with negative input offset") {
+    runQueryAndCompare(
+      "select lag(l_orderkey, -2) over" +
+        " (partition by l_suppkey order by l_orderkey) from lineitem") {
+      checkOperatorMatch[WindowExecTransformer]
+    }
+
+    runQueryAndCompare(
+      "select lead(l_orderkey, -2) over" +
+        " (partition by l_suppkey order by l_orderkey) from lineitem") {
+      checkOperatorMatch[WindowExecTransformer]
+    }
+  }
+
+}
diff --git 
a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
 
b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
index b257f8a6c..dc30f0559 100644
--- 
a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
+++ 
b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
@@ -25,7 +25,6 @@ import org.apache.spark.SparkConf
 import org.junit.Assert
 
 class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite {
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
diff --git 
a/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala
 
b/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala
index 3a6d2e2c0..10cefa1d2 100644
--- 
a/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala
+++ 
b/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala
@@ -39,7 +39,6 @@ abstract class WholeStageTransformerSuite
   with SharedSparkSession
   with AdaptiveSparkPlanHelper {
 
-  protected val backend: String
   protected val resourcePath: String
   protected val fileFormat: String
   protected val logLevel: String = "WARN"
diff --git 
a/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala 
b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala
index accb76472..0632d3e24 100644
--- 
a/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala
+++ 
b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala
@@ -25,7 +25,6 @@ import scala.collection.JavaConverters._
 class VeloxDeltaSuite extends WholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 
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 9dc933562..ec8f63044 100644
--- 
a/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
+++ 
b/gluten-iceberg/src/test/scala/io/glutenproject/execution/VeloxIcebergSuite.scala
@@ -24,7 +24,6 @@ import org.apache.spark.sql.Row
 class VeloxIcebergSuite extends WholeStageTransformerSuite {
 
   protected val rootPath: String = getClass.getResource("/").getPath
-  override protected val backend: String = "velox"
   override protected val resourcePath: String = "/tpch-data-parquet-velox"
   override protected val fileFormat: String = "parquet"
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@gluten.apache.org
For additional commands, e-mail: commits-h...@gluten.apache.org


Reply via email to