diff --git a/.github/workflows/util/install-spark-resources.sh b/.github/workflows/util/install-spark-resources.sh index dd5354239b1c..e1677bb6a137 100755 --- a/.github/workflows/util/install-spark-resources.sh +++ b/.github/workflows/util/install-spark-resources.sh @@ -113,6 +113,11 @@ case "$1" in cd ${INSTALL_DIR} && \ install_spark "3.5.5" "3" "2.13" ;; +4.0) + # Spark-4.0, scala 2.12 // using 2.12 as a hack as 4.0 does not have 2.13 suffix + cd ${INSTALL_DIR} && \ + install_spark "4.0.1" "3" "2.12" + ;; *) echo "Spark version is expected to be specified." exit 1 diff --git a/.github/workflows/velox_backend_x86.yml b/.github/workflows/velox_backend_x86.yml index e7afd1b189ca..4c56b2f0642f 100644 --- a/.github/workflows/velox_backend_x86.yml +++ b/.github/workflows/velox_backend_x86.yml @@ -1297,3 +1297,105 @@ jobs: df -a bash dev/buildbundle-veloxbe.sh --run_setup_script=OFF --build_arrow=OFF --spark_version=3.4 --enable_gpu=ON ccache -s + + spark-test-spark40: + needs: build-native-lib-centos-7 + runs-on: ubuntu-22.04 + container: apache/gluten:centos-8-jdk17 + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v4 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases + - name: Download Arrow Jars + uses: actions/download-artifact@v4 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare + run: | + dnf module -y install python39 && \ + alternatives --set python3 /usr/bin/python3.9 && \ + pip3 install setuptools==77.0.3 && \ + pip3 install pyspark==3.5.5 cython && \ + pip3 install pandas==2.2.3 pyarrow==20.0.0 + - name: Prepare Spark Resources for Spark 4.0.1 #TODO remove after image update + run: | + rm -rf /opt/shims/spark40 + bash .github/workflows/util/install-spark-resources.sh 4.0 + mv /opt/shims/spark40/spark_home/assembly/target/scala-2.12 /opt/shims/spark40/spark_home/assembly/target/scala-2.13 + - name: Build and Run unit test for Spark 4.0.0 with scala-2.13 (other tests) + run: | + cd $GITHUB_WORKSPACE/ + export SPARK_SCALA_VERSION=2.13 + yum install -y java-17-openjdk-devel + export JAVA_HOME=/usr/lib/jvm/java-17-openjdk + export PATH=$JAVA_HOME/bin:$PATH + java -version + $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox \ + -Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark40/spark_home/" \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + - name: Upload test report + if: always() + uses: actions/upload-artifact@v4 + with: + name: ${{ github.job }}-report + path: '**/surefire-reports/TEST-*.xml' + - name: Upload unit tests log files + if: ${{ !success() }} + uses: actions/upload-artifact@v4 + with: + name: ${{ github.job }}-test-log + path: | + **/target/*.log + **/gluten-ut/**/hs_err_*.log + **/gluten-ut/**/core.* + + spark-test-spark40-slow: + needs: build-native-lib-centos-7 + runs-on: ubuntu-22.04 + container: apache/gluten:centos-8-jdk17 + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v4 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases + - name: Download Arrow Jars + uses: actions/download-artifact@v4 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 4.0.1 #TODO remove after image update + run: | + rm -rf /opt/shims/spark40 + bash .github/workflows/util/install-spark-resources.sh 4.0 + mv /opt/shims/spark40/spark_home/assembly/target/scala-2.12 /opt/shims/spark40/spark_home/assembly/target/scala-2.13 + - name: Build and Run unit test for Spark 4.0 (slow tests) + run: | + cd $GITHUB_WORKSPACE/ + yum install -y java-17-openjdk-devel + export JAVA_HOME=/usr/lib/jvm/java-17-openjdk + export PATH=$JAVA_HOME/bin:$PATH + java -version + $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark40/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + - name: Upload test report + if: always() + uses: actions/upload-artifact@v4 + with: + name: ${{ github.job }}-report + path: '**/surefire-reports/TEST-*.xml' + - name: Upload unit tests log files + if: ${{ !success() }} + uses: actions/upload-artifact@v4 + with: + name: ${{ github.job }}-test-log + path: | + **/target/*.log + **/gluten-ut/**/hs_err_*.log + **/gluten-ut/**/core.* diff --git a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala index 6d83d59d9e76..7ef1a1c06d29 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala @@ -71,7 +71,8 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { } } - test("Test plan json non-empty - AQE on") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("Test plan json non-empty - AQE on", "3.5") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true") { diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index b6578c0ea534..ad18d6842aad 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -135,7 +135,8 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa checkLengthAndPlan(df, 2) } - test("is_not_null") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("is_not_null", "3.5") { val df = runQueryAndCompare( "select l_orderkey from lineitem where l_comment is not null " + "and l_orderkey = 1") { _ => } @@ -175,7 +176,8 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa checkLengthAndPlan(df, 0) } - test("and pushdown") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("and pushdown", "3.5") { val df = runQueryAndCompare( "select l_orderkey from lineitem where l_orderkey > 2 " + "and l_orderkey = 1") { _ => } @@ -351,7 +353,8 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa checkLengthAndPlan(df, 7) } - test("window expression") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("window expression", "3.5") { runQueryAndCompare( "select max(l_partkey) over" + " (partition by l_suppkey order by l_commitdate" + diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxHashJoinSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxHashJoinSuite.scala index fe2ae6ba71f3..53f44a2ccc81 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxHashJoinSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxHashJoinSuite.scala @@ -94,7 +94,10 @@ class VeloxHashJoinSuite extends VeloxWholeStageTransformerSuite { val wholeStages = plan.collect { case wst: WholeStageTransformer => wst } if (SparkShimLoader.getSparkVersion.startsWith("3.2.")) { assert(wholeStages.length == 1) - } else if (SparkShimLoader.getSparkVersion.startsWith("3.5.")) { + } else if ( + SparkShimLoader.getSparkVersion.startsWith("3.5.") || + SparkShimLoader.getSparkVersion.startsWith("4.0.") + ) { assert(wholeStages.length == 5) } else { assert(wholeStages.length == 3) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/python/ArrowEvalPythonExecSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/python/ArrowEvalPythonExecSuite.scala index c2a191a20d0b..c857dced844f 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/python/ArrowEvalPythonExecSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/python/ArrowEvalPythonExecSuite.scala @@ -39,7 +39,8 @@ class ArrowEvalPythonExecSuite extends WholeStageTransformerSuite { .set("spark.executor.cores", "1") } - test("arrow_udf test: without projection") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("arrow_udf test: without projection", "3.5") { lazy val base = Seq(("1", 1), ("1", 2), ("2", 1), ("2", 2), ("3", 1), ("3", 2), ("0", 1), ("3", 0)) .toDF("a", "b") @@ -59,7 +60,8 @@ class ArrowEvalPythonExecSuite extends WholeStageTransformerSuite { checkAnswer(df2, expected) } - test("arrow_udf test: with unrelated projection") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("arrow_udf test: with unrelated projection", "3.5") { lazy val base = Seq(("1", 1), ("1", 2), ("2", 1), ("2", 2), ("3", 1), ("3", 2), ("0", 1), ("3", 0)) .toDF("a", "b") @@ -79,7 +81,8 @@ class ArrowEvalPythonExecSuite extends WholeStageTransformerSuite { checkAnswer(df, expected) } - test("arrow_udf test: with preprojection") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("arrow_udf test: with preprojection", "3.5") { lazy val base = Seq(("1", 1), ("1", 2), ("2", 1), ("2", 2), ("3", 1), ("3", 2), ("0", 1), ("3", 0)) .toDF("a", "b") diff --git a/backends-velox/src/test/scala/org/apache/gluten/functions/ArithmeticAnsiValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/functions/ArithmeticAnsiValidateSuite.scala index b3ee4d432146..5b9ee0e35a31 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/functions/ArithmeticAnsiValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/functions/ArithmeticAnsiValidateSuite.scala @@ -33,7 +33,8 @@ class ArithmeticAnsiValidateSuite extends FunctionsValidateSuite { .set(SQLConf.ANSI_ENABLED.key, "true") } - test("add") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("add", "3.5") { runQueryAndCompare("SELECT int_field1 + 100 FROM datatab WHERE int_field1 IS NOT NULL") { checkGlutenOperatorMatch[ProjectExecTransformer] } @@ -48,7 +49,8 @@ class ArithmeticAnsiValidateSuite extends FunctionsValidateSuite { } } - test("multiply") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("multiply", "3.5") { runQueryAndCompare("SELECT int_field1 * 2 FROM datatab WHERE int_field1 IS NOT NULL") { checkGlutenOperatorMatch[ProjectExecTransformer] } diff --git a/backends-velox/src/test/scala/org/apache/gluten/functions/DateFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/functions/DateFunctionsValidateSuite.scala index c3e016587335..8e0660bb86a2 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/functions/DateFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/functions/DateFunctionsValidateSuite.scala @@ -278,7 +278,8 @@ abstract class DateFunctionsValidateSuite extends FunctionsValidateSuite { } } - testWithMinSparkVersion("timestampadd", "3.3") { + // TODO: fix on spark-4.0 + testWithRangeSparkVersion("timestampadd", "3.3", "3.5") { withTempPath { path => val ts = Timestamp.valueOf("2020-02-29 00:00:00.500") diff --git a/backends-velox/src/test/scala/org/apache/gluten/functions/JsonFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/functions/JsonFunctionsValidateSuite.scala index 7331054c4818..f7b0f286a71b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/functions/JsonFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/functions/JsonFunctionsValidateSuite.scala @@ -59,7 +59,8 @@ class JsonFunctionsValidateSuite extends FunctionsValidateSuite { } } - test("json_array_length") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("json_array_length", "3.5") { runQueryAndCompare( s"select *, json_array_length(string_field1) " + s"from datatab limit 5")(checkGlutenOperatorMatch[ProjectExecTransformer]) @@ -348,7 +349,8 @@ class JsonFunctionsValidateSuite extends FunctionsValidateSuite { } } - test("json_object_keys") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("json_object_keys", "3.5") { withTempPath { path => Seq[String]( @@ -378,7 +380,8 @@ class JsonFunctionsValidateSuite extends FunctionsValidateSuite { } } - test("to_json function") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("to_json function", "3.5") { withTable("t") { spark.sql( """ diff --git a/backends-velox/src/test/scala/org/apache/gluten/functions/ScalarFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/functions/ScalarFunctionsValidateSuite.scala index f6be6061316a..911ef4a568ba 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/functions/ScalarFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/functions/ScalarFunctionsValidateSuite.scala @@ -522,7 +522,8 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateSuite { } } - test("raise_error, assert_true") { + // TODO: fix on spark-4.0 + testWithMaxSparkVersion("raise_error, assert_true", "3.5") { runQueryAndCompare("""SELECT assert_true(l_orderkey >= 1), l_orderkey | from lineitem limit 100""".stripMargin) { checkGlutenOperatorMatch[ProjectExecTransformer] @@ -555,7 +556,7 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateSuite { } } - test("version") { + testWithMaxSparkVersion("version", "3.5") { runQueryAndCompare("""SELECT version() from lineitem limit 10""".stripMargin) { checkGlutenOperatorMatch[ProjectExecTransformer] } @@ -1097,7 +1098,8 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateSuite { } } - testWithMinSparkVersion("try_cast", "3.4") { + // TODO: fix on spark-4.0 + testWithRangeSparkVersion("try_cast", "3.4", "3.5") { withTempView("try_cast_table") { withTempPath { path => diff --git a/dev/docker/Dockerfile.centos8-dynamic-build b/dev/docker/Dockerfile.centos8-dynamic-build index 4f81ed557e97..9c4033de894c 100644 --- a/dev/docker/Dockerfile.centos8-dynamic-build +++ b/dev/docker/Dockerfile.centos8-dynamic-build @@ -47,11 +47,12 @@ RUN set -ex; \ wget -nv https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz -P /opt/; \ git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten; \ cd /opt/gluten/.github/workflows/util/; \ - ./install-spark-resources.sh 3.2; \ - ./install-spark-resources.sh 3.3; \ - ./install-spark-resources.sh 3.4; \ - ./install-spark-resources.sh 3.5; \ - ./install-spark-resources.sh 3.5-scala2.13; \ + ./install_spark_resources.sh 3.2; \ + ./install_spark_resources.sh 3.3; \ + ./install_spark_resources.sh 3.4; \ + ./install_spark_resources.sh 3.5; \ + ./install_spark_resources.sh 3.5-scala2.13; \ + ./install_spark_resources.sh 4.0; \ if [ "$(uname -m)" = "aarch64" ]; then \ export CPU_TARGET="aarch64"; \ fi; \ diff --git a/dev/docker/Dockerfile.centos9-dynamic-build b/dev/docker/Dockerfile.centos9-dynamic-build index 2a4db45e4c01..600de3536a04 100644 --- a/dev/docker/Dockerfile.centos9-dynamic-build +++ b/dev/docker/Dockerfile.centos9-dynamic-build @@ -45,11 +45,12 @@ RUN set -ex; \ wget -nv https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz -P /opt/; \ git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten; \ cd /opt/gluten/.github/workflows/util/; \ - ./install-spark-resources.sh 3.2; \ - ./install-spark-resources.sh 3.3; \ - ./install-spark-resources.sh 3.4; \ - ./install-spark-resources.sh 3.5; \ - ./install-spark-resources.sh 3.5-scala2.13; \ + ./install_spark_resources.sh 3.2; \ + ./install_spark_resources.sh 3.3; \ + ./install_spark_resources.sh 3.4; \ + ./install_spark_resources.sh 3.5; \ + ./install_spark_resources.sh 3.5-scala2.13; \ + ./install_spark_resources.sh 4.0; \ if [ "$(uname -m)" = "aarch64" ]; then \ export CPU_TARGET="aarch64"; \ fi; \ diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala index 7a02698cdc78..efa0c63dca52 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala @@ -24,6 +24,7 @@ object SparkVersionUtil { val lteSpark33: Boolean = lteSpark32 || eqSpark33 val gteSpark33: Boolean = comparedWithSpark33 >= 0 val gteSpark35: Boolean = comparedWithSpark35 >= 0 + val gteSpark40: Boolean = compareMajorMinorVersion((4, 0)) >= 0 // Returns X. X < 0 if one < other, x == 0 if one == other, x > 0 if one > other. def compareMajorMinorVersion(other: (Int, Int)): Int = { diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index 4d292ea13e76..fae13c8493ec 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -33,8 +33,10 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan, SQLExecution, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.SparkVersionUtil import org.junit.Assert import org.scalatest.Assertions @@ -45,7 +47,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.reflect.runtime.universe -abstract class GlutenQueryTest extends PlanTest { +abstract class GlutenQueryTest extends PlanTest with AdaptiveSparkPlanHelper { // TODO: remove this if we can suppress unused import error. locally { @@ -356,7 +358,12 @@ abstract class GlutenQueryTest extends PlanTest { } private def getExecutedPlan(plan: SparkPlan): Seq[SparkPlan] = { - val subTree = plan match { + val stripPlan = if (SparkVersionUtil.gteSpark40) { + stripAQEPlan(plan) + } else { + plan + } + val subTree = stripPlan match { case exec: AdaptiveSparkPlanExec => getExecutedPlan(exec.executedPlan) case cmd: CommandResultExec => diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala index 67b6531df0e1..cc564ffa3890 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/DummyFilterColmnarHelper.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.classic.ClassicDataset import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy, UnaryExecNode} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.vectorized.ColumnarBatch @@ -59,6 +61,10 @@ case class DummyFilterColumnarExec(child: SparkPlan) extends UnaryExecNode { } object DummyFilterColumnarStrategy extends SparkStrategy { + // TODO: remove this if we can suppress unused import error. + locally { + new ColumnConstructorExt(Column) + } override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case r: DummyFilterColumnar => DummyFilterColumnarExec(planLater(r.child)) :: Nil @@ -73,7 +79,7 @@ object DummyFilterColmnarHelper { case p => p } - Dataset.ofRows(spark, modifiedPlan) + ClassicDataset.ofRows(spark, modifiedPlan) } def withSession(builders: Seq[SparkSessionExtensionsProvider])(f: SparkSession => Unit): Unit = { diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsTrait.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsTrait.scala index 7ca91786a408..ef0b46c69f3b 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsTrait.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenSQLTestsTrait.scala @@ -20,6 +20,7 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.util.{sideBySide, stackTraceToString} +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.SQLExecution import org.apache.commons.io.FileUtils @@ -81,6 +82,10 @@ trait GlutenSQLTestsTrait extends QueryTest with GlutenSQLTestsBaseTrait { } object GlutenQueryTestUtil extends Assertions { + // TODO: remove this if we can suppress unused import error. + locally { + new ColumnConstructorExt(Column) + } /** * Runs the plan and makes sure the answer matches the expected result. diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala index 22c2280cb2f1..14455a7e97c4 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala @@ -28,6 +28,8 @@ import org.apache.spark.sql.catalyst.analysis.ResolveTimeZone import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, MapData, TypeUtils} +import org.apache.spark.sql.classic.ClassicColumn +import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -42,7 +44,10 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer trait GlutenTestsTrait extends GlutenTestsCommonTrait { - + // TODO: remove this if we can suppress unused import error. + locally { + new ColumnConstructorExt(Column) + } override def beforeAll(): Unit = { // prepare working paths val basePathDir = new File(basePath) @@ -244,7 +249,7 @@ trait GlutenTestsTrait extends GlutenTestsCommonTrait { val empData = Seq(Row(1)) _spark.createDataFrame(_spark.sparkContext.parallelize(empData), schema) } - val resultDF = df.select(Column(expression)) + val resultDF = df.select(ClassicColumn(expression)) val result = resultDF.collect() TestStats.testUnitNumber = TestStats.testUnitNumber + 1 if ( diff --git a/gluten-ut/pom.xml b/gluten-ut/pom.xml index 2726533b8a7c..70e0da626cf1 100644 --- a/gluten-ut/pom.xml +++ b/gluten-ut/pom.xml @@ -219,5 +219,11 @@ spark35 + + spark-4.0 + + spark40 + + diff --git a/gluten-ut/spark40/pom.xml b/gluten-ut/spark40/pom.xml new file mode 100644 index 000000000000..e882f79fea4c --- /dev/null +++ b/gluten-ut/spark40/pom.xml @@ -0,0 +1,245 @@ + + + 4.0.0 + + org.apache.gluten + gluten-ut + 1.6.0-SNAPSHOT + ../pom.xml + + + gluten-ut-spark40 + jar + Gluten Unit Test Spark40 + + + + org.apache.gluten + gluten-ut-common + ${project.version} + test-jar + compile + + + org.apache.parquet + parquet-column + 1.15.2 + tests + test + + + org.apache.parquet + parquet-common + 1.15.2 + tests + test + + + + + + + org.apache.maven.plugins + maven-resources-plugin + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.scalatest + scalatest-maven-plugin + + . + + + + test + + test + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + backends-clickhouse + + false + + + + org.apache.gluten + backends-clickhouse + ${project.version} + test + + + org.apache.celeborn + celeborn-client-spark-${spark.major.version}-shaded_${scala.binary.version} + ${celeborn.version} + test + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + provided + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + org.apache.arrow + arrow-vector + ${arrow.version} + provided + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + + add-test-source + + generate-sources + + + src/test/backends-clickhouse + + + + + + + + + + backends-velox + + false + + + 2.0.16 + 2.24.3 + + + + org.junit.jupiter + junit-jupiter + 5.9.3 + test + + + net.aichler + jupiter-interface + 0.11.1 + test + + + org.apache.gluten + backends-velox + ${project.version} + test + + + + org.slf4j + slf4j-api + ${slf4j.version} + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + ${log4j.version} + test + + + com.h2database + h2 + 2.2.220 + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + + add-test-source + + generate-sources + + + src/test/backends-velox + + + + + + + + + + diff --git a/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala b/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala new file mode 100644 index 000000000000..4258cd891a5a --- /dev/null +++ b/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala @@ -0,0 +1,27 @@ +/* + * 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.gluten + +import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, SparkPlan} + +trait GlutenColumnarWriteTestSupport { + + def checkWriteFilesAndGetChild(sparkPlan: SparkPlan): SparkPlan = { + assert(sparkPlan.isInstanceOf[ColumnarWriteFilesExec]) + sparkPlan.asInstanceOf[ColumnarWriteFilesExec].child + } +} diff --git a/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala b/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala new file mode 100644 index 000000000000..5b84888376aa --- /dev/null +++ b/gluten-ut/spark40/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala @@ -0,0 +1,28 @@ +/* + * 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.gluten.execution.parquet + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.datasources.parquet.GlutenParquetV1FilterSuite + + +/** testing use_local_format parquet reader. **/ +class GlutenParquetV1FilterSuite2 extends GlutenParquetV1FilterSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") +} diff --git a/gluten-ut/spark40/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala b/gluten-ut/spark40/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala new file mode 100644 index 000000000000..68c0e1c932b5 --- /dev/null +++ b/gluten-ut/spark40/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala @@ -0,0 +1,27 @@ +/* + * 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.gluten + +import org.apache.spark.sql.execution.{SparkPlan, ColumnarWriteFilesExec} + +trait GlutenColumnarWriteTestSupport { + + def checkWriteFilesAndGetChild(sparkPlan: SparkPlan): SparkPlan = { + assert(sparkPlan.isInstanceOf[ColumnarWriteFilesExec]) + sparkPlan.asInstanceOf[ColumnarWriteFilesExec].child + } +} diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 000000000000..b773396c050d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,96 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- group by ordinal followed by order by +select a, count(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select count(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, count(b) from data group by k, 1; + +-- can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, 2); + +-- mixed cases: can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, b); + +-- can use ordinal with cube +select a, b, count(1) from data group by 1, 2 with cube; + +-- can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, 2); + +-- mixed cases: can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, b); + +-- can use ordinal with rollup +select a, b, count(1) from data group by 1, 2 with rollup; + +-- can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)); + +-- mixed cases: can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)); + +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)); + +-- range error +select a, b, count(1) from data group by a, -1; + +select a, b, count(1) from data group by a, 3; + +select a, b, count(1) from data group by cube(-1, 2); + +select a, b, count(1) from data group by cube(1, 3); + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql new file mode 100644 index 000000000000..c35cdb0de271 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql @@ -0,0 +1,266 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- struct() in group by +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa); + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); + +-- SPARK-34581: Don't optimize out grouping expressions from aggregate expressions without aggregate function +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL; + +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL; + + +-- Histogram aggregates with different numeric input types +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col); + +-- SPARK-27974: Support ANSI Aggregate Function: array_agg +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col); +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a; + + +SELECT mode(a), mode(b) FROM testData; +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a; + + +-- SPARK-44846: PushFoldableIntoBranches in complex grouping expressions cause bindReference error +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c; diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql new file mode 100644 index 000000000000..c7cb5bf1117a --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql @@ -0,0 +1,52 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x); + +-- SPARK-37613: Support ANSI Aggregate Function: regr_count +SELECT regr_count(y, x) FROM testRegression; +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k; +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37613: Support ANSI Aggregate Function: regr_r2 +SELECT regr_r2(y, x) FROM testRegression; +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k; +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression; +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37672: Support ANSI Aggregate Function: regr_sxx +SELECT regr_sxx(y, x) FROM testRegression; +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37681: Support ANSI Aggregate Function: regr_sxy +SELECT regr_sxy(y, x) FROM testRegression; +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37702: Support ANSI Aggregate Function: regr_syy +SELECT regr_syy(y, x) FROM testRegression; +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-39230: Support ANSI Aggregate Function: regr_slope +SELECT regr_slope(y, x) FROM testRegression; +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37623: Support ANSI Aggregate Function: regr_intercept +SELECT regr_intercept(y, x) FROM testRegression; +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql new file mode 100644 index 000000000000..907ff33000d8 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,22 @@ +-- test for misc functions + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); + +-- Spark-32793: Rewrite AssertTrue with RaiseError +SELECT assert_true(true), assert_true(boolean(1)); +SELECT assert_true(false); +SELECT assert_true(boolean(0)); +SELECT assert_true(null); +SELECT assert_true(boolean(null)); +SELECT assert_true(false, 'custom error message'); + +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v); +SELECT raise_error('error message'); +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc; diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql new file mode 100644 index 000000000000..a1aae7b8759d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql @@ -0,0 +1,17 @@ +-- rand with the seed 0 +SELECT rand(0); +SELECT rand(cast(3 / 7 AS int)); +SELECT rand(NULL); +SELECT rand(cast(NULL AS int)); + +-- rand unsupported data type +SELECT rand(1.0); + +-- randn with the seed 0 +SELECT randn(0L); +SELECT randn(cast(3 / 7 AS long)); +SELECT randn(NULL); +SELECT randn(cast(NULL AS long)); + +-- randn unsupported data type +SELECT rand('1') diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 000000000000..79d427bc2099 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,126 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call with invalid number of arguments +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); + +-- range call with incompatible type +select * from range(array(1, 2, 3)); + +-- range call with illegal step +select * from range(0, 5, 0); + +-- range call with a mixed-case function name +select * from RaNgE(2); + +-- range call with alias +select i from range(0, 2) t(i); + +-- explode +select * from explode(array(1, 2)); +select * from explode(map('a', 1, 'b', 2)); + +-- explode with empty values +select * from explode(array()); +select * from explode(map()); + +-- explode with column aliases +select * from explode(array(1, 2)) t(c1); +select * from explode(map('a', 1, 'b', 2)) t(k, v); + +-- explode with non-deterministic values +select * from explode(array(rand(0))); + +-- explode with erroneous input +select * from explode(null); +select * from explode(null) t(c1); +select * from explode(1); +select * from explode(1, 2); +select * from explode(explode(array(1))); +select * from explode(array(1, 2)) t(c1, c2); + +-- explode_outer +select * from explode_outer(array(1, 2)); +select * from explode_outer(map('a', 1, 'b', 2)); +select * from explode_outer(array()); +select * from explode_outer(map()); + +-- table-valued functions with join +select * from range(2) join explode(array(1, 2)); +select * from range(2) join explode_outer(array()); + +-- inline +select * from inline(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y); +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- inline with erroneous input +select * from inline(null); +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c); + +-- inline_outer +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- posexplode +select * from posexplode(array()); +select * from posexplode(array(1, 2)); +select * from posexplode(array(1, 2)) t(pos, x); +select * from posexplode(map()); +select * from posexplode(map('a', 1, 'b', 2)); +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v); + +-- posexplode with erroneous input +select * from posexplode(1); +select * from posexplode(1, 2); +select * from posexplode(explode(array(1))); +select * from posexplode(array(1, 2)) t(x); + +-- posexplode +select * from posexplode_outer(array()); +select * from posexplode_outer(array(1, 2)); +select * from posexplode_outer(map()); +select * from posexplode_outer(map('a', 1, 'b', 2)); + +-- json_tuple +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y); +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b'); + +-- json_tuple with erroneous input +select * from json_tuple(); +select * from json_tuple('{"a": 1}'); +select * from json_tuple('{"a": 1}', 1); +select * from json_tuple('{"a": 1}', null); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x); + +-- stack +select * from stack(1, 1, 2, 3); +select * from stack(2, 1, 2, 3); +select * from stack(3, 1, 2, 3) t(x); +select * from stack(4, 1, 2, 3) t(x); +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c); +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c); + +-- stack with erroneous input +select * from stack(); +select * from stack(2, 1, 2, 3) t(a, b, c); +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b'); +select * from stack(2, explode(array(1, 2, 3))); diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql new file mode 100644 index 000000000000..0cc57c97b020 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql @@ -0,0 +1,156 @@ +-- This test file was converted from group-by.sql. +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT udf(a), udf(COUNT(b)) FROM testData; +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a; +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b; +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a); + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate with complex GroupBy expressions. +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b; +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1; +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); + +-- Aggregate with nulls. +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a); + +-- Aliases in SELECT could be used in GROUP BY +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k; +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1; + +-- Aggregate functions cannot be used in GROUP BY +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a); + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a); + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT udf(COUNT(1)) FROM testData WHERE false; +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT udf(1) FROM range(10) HAVING true; + +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0; + +SELECT udf(id) FROM range(10) HAVING id > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k); + +-- having +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k); + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)); + +-- input type checking Int +SELECT every(udf(1)); + +-- input type checking Short +SELECT some(udf(1S)); + +-- input type checking Long +SELECT any(udf(1L)); + +-- input type checking String +SELECT udf(every("true")); + +-- every/some/any aggregates are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L; +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L; +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 000000000000..b968b4e09fac --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,524 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, sum(b) from data group by 1 +-- !query schema +struct +-- !query output +1 3 +2 3 +3 3 + + +-- !query +select 1, 2, sum(b) from data group by 1, 2 +-- !query schema +struct<1:int,2:int,sum(b):bigint> +-- !query output +1 2 9 + + +-- !query +select a, 1, sum(b) from data group by a, 1 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, 1, sum(b) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, b + 2, count(2) from data group by a, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select sum(b) from data group by 1 + 0 +-- !query schema +struct +-- !query output +9 + + +-- !query +select a, b from data group by -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b from data group by 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] +} + + +-- !query +select a, b from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) + 2 from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query schema +struct +-- !query output +1 0.5488135024422883 1 +1 0.7151893651681639 2 +2 0.5448831775801376 2 +2 0.6027633705776989 1 +3 0.4236547969336536 1 +3 0.6458941151817286 2 + + +-- !query +select * from data group by a, b, 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "STAR_GROUP_BY_POS", + "sqlState" : "0A000" +} + + +-- !query +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select a, a AS k, count(b) from data group by k, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 2 2 +3 3 2 + + +-- !query +select a, b, count(1) from data group by cube(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by cube(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with cube +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with rollup +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by a, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(-1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(1, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.groupByOrdinal false + + +-- !query +select sum(b) from data group by -1 +-- !query schema +struct +-- !query output +9 diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out new file mode 100644 index 000000000000..a4a3f76fa6a7 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out @@ -0,0 +1,1123 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT a, COUNT(b) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b):bigint> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa) +-- !query schema +struct +-- !query output +2 +2 +2 +3 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" + } ] +} + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "count(testdata.b)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "COUNT(b)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 48, + "fragment" : "k" + } ] +} + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT id FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(max(id) > 0)\"", + "expressionList" : "max(id)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0" + } ] +} + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true false true + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct +-- !query output +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "every(1)" + } ] +} + + +-- !query +SELECT some(1S) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "some(1S)" + } ] +} + + +-- !query +SELECT any(1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "any(1L)" + } ] +} + + +-- !query +SELECT every("true") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT bool_and(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_and(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_and(1.0)" + } ] +} + + +-- !query +SELECT bool_or(1.0D) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_or(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_or(1.0D)" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(NOT (a IS NULL)):boolean,c:bigint> +-- !query output +false 2 +true 7 + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint> +-- !query output +0.5488135024422883 7 +1.0 2 + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query schema +struct>,histogram_3:array>,histogram_5:array>,histogram_10:array>> +-- !query output +[{"x":12,"y":26.0},{"x":38,"y":24.0}] [{"x":9,"y":20.0},{"x":25,"y":11.0},{"x":40,"y":19.0}] [{"x":5,"y":11.0},{"x":14,"y":8.0},{"x":22,"y":7.0},{"x":30,"y":10.0},{"x":43,"y":14.0}] [{"x":3,"y":6.0},{"x":8,"y":6.0},{"x":13,"y":4.0},{"x":17,"y":3.0},{"x":20,"y":4.0},{"x":25,"y":6.0},{"x":31,"y":7.0},{"x":39,"y":5.0},{"x":43,"y":4.0},{"x":48,"y":5.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":2017-03-01 00:00:00,"y":1.0},{"x":2017-04-01 00:00:00,"y":1.0},{"x":2017-05-01 00:00:00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":100-0,"y":1.0},{"x":110-0,"y":1.0},{"x":120-0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":12 20:04:00.000000000,"y":1.0},{"x":12 21:04:00.000000000,"y":1.0},{"x":12 22:04:00.000000000,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query schema +struct,collect_list(col):array> +-- !query output +[1,2,1] [1,2,1] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query schema +struct,collect_list(b):array> +-- !query output +1 [4,4] [4,4] +2 [3,4] [3,4] + + +-- !query +SELECT mode(a), mode(b) FROM testData +-- !query schema +struct +-- !query output +3 1 + + +-- !query +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a +-- !query schema +struct +-- !query output +NULL 1 +1 1 +2 1 +3 1 + + +-- !query +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c +-- !query schema +struct +-- !query output +0 +2 diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out new file mode 100644 index 000000000000..a73347a2f7f5 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out @@ -0,0 +1,276 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 1 0 +2 4 3 + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 0 0 +2 3 3 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL 10.0 NULL NULL +2 22.666666666666668 21.25 22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL NULL NULL +2 22.666666666666668 20.0 22.666666666666668 20.0 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 288.66666666666663 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 240.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 200.0 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 0.8314087759815244 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression +-- !query schema +struct +-- !query output +1.1547344110854487 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +1.1547344110854487 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 1.1547344110854487 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 1.1547344110854487 diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out new file mode 100644 index 000000000000..d6d1289a5adb --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,134 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select typeof(null) +-- !query schema +struct +-- !query output +void + + +-- !query +select typeof(true) +-- !query schema +struct +-- !query output +boolean + + +-- !query +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query schema +struct +-- !query output +tinyint smallint int bigint + + +-- !query +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query schema +struct +-- !query output +float double decimal(2,1) + + +-- !query +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query schema +struct +-- !query output +date timestamp interval day + + +-- !query +select typeof(x'ABCD'), typeof('SPARK') +-- !query schema +struct +-- !query output +binary string + + +-- !query +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query schema +struct +-- !query output +array map struct + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'false' is not true! + + +-- !query +SELECT assert_true(boolean(0)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(0 as boolean)' is not true! + + +-- !query +SELECT assert_true(null) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'null' is not true! + + +-- !query +SELECT assert_true(boolean(null)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(null as boolean)' is not true! + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +custom error message + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT raise_error('error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +error message + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +too big: 8 diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out new file mode 100644 index 000000000000..17e6f871b9c5 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out @@ -0,0 +1,115 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT rand(0) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(3 / 7 AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(NULL) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(NULL AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand(1.0)" + } ] +} + + +-- !query +SELECT randn(0L) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(3 / 7 AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(NULL) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(NULL AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT rand('1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand('1')" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 000000000000..0d5675fa6fde --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,1017 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from dummy(3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "messageParameters" : { + "name" : "`dummy`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 22, + "fragment" : "dummy(3)" + } ] +} + + +-- !query +select * from range(6 + cos(3)) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 + + +-- !query +select * from range(5, 10) +-- !query schema +struct +-- !query output +5 +6 +7 +8 +9 + + +-- !query +select * from range(0, 10, 2) +-- !query schema +struct +-- !query output +0 +2 +4 +6 +8 + + +-- !query +select * from range(0, 10, 1, 200) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +select * from range(1, 1, 1, 1, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4]", + "functionName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(1, 1, 1, 1, 1)" + } ] +} + + +-- !query +select * from range(1, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(1, null)" + } ] +} + + +-- !query +select * from range(array(1, 2, 3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"array(1, 2, 3)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 35, + "fragment" : "range(array(1, 2, 3))" + } ] +} + + +-- !query +select * from range(0, 5, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(0, 5, 0)" + } ] +} + + +-- !query +select * from RaNgE(2) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select i from range(0, 2) t(i) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select * from explode(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(array(1, 2)) t(c1) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) t(k, v) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array(rand(0))) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +select * from explode(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(null)" + } ] +} + + +-- !query +select * from explode(null) t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "explode(null) t(c1)" + } ] +} + + +-- !query +select * from explode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "explode(1)" + } ] +} + + +-- !query +select * from explode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`explode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(1, 2)" + } ] +} + + +-- !query +select * from explode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "explode(explode(array(1)))" + } ] +} + + +-- !query +select * from explode(array(1, 2)) t(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "2", + "funcName" : "`explode`", + "outColsNum" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 44, + "fragment" : "explode(array(1, 2)) t(c1, c2)" + } ] +} + + +-- !query +select * from explode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode_outer(array()) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select * from explode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from range(2) join explode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 1 +1 2 + + +-- !query +select * from range(2) join explode_outer(array()) +-- !query schema +struct +-- !query output +0 NULL +1 NULL + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output + + + +-- !query +select * from inline(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"inline(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 26, + "fragment" : "inline(null)" + } ] +} + + +-- !query +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`inline`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 66, + "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)" + } ] +} + + +-- !query +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(array(1, 2)) t(pos, x) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"posexplode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "posexplode(1)" + } ] +} + + +-- !query +select * from posexplode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`posexplode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 30, + "fragment" : "posexplode(1, 2)" + } ] +} + + +-- !query +select * from posexplode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"posexplode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 43, + "fragment" : "posexplode(explode(array(1)))" + } ] +} + + +-- !query +select * from posexplode(array(1, 2)) t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`posexplode`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "posexplode(array(1, 2)) t(x)" + } ] +} + + +-- !query +select * from posexplode_outer(array()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select * from posexplode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c') +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a') +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b') +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from json_tuple() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 39, + "fragment" : "json_tuple('{\"a\": 1}', 1)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1}', null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "json_tuple('{\"a\": 1}', null)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`json_tuple`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 62, + "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)" + } ] +} + + +-- !query +select * from stack(1, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 3 + + +-- !query +select * from stack(2, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 +3 NULL + + +-- !query +select * from stack(3, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from stack(4, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 a +2 2.2 b + + +-- !query +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 NULL +2 NULL b + + +-- !query +select * from stack() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`stack`" + } +} + + +-- !query +select * from stack(2, 1, 2, 3) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`stack`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "stack(2, 1, 2, 3) t(a, b, c)" + } ] +} + + +-- !query +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "columnIndex" : "1", + "leftParamIndex" : "2", + "leftType" : "\"STRING\"", + "rightParamIndex" : "5", + "rightType" : "\"DECIMAL(2,1)\"", + "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 50, + "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')" + } ] +} + + +-- !query +select * from stack(2, explode(array(1, 2, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"stack(2, explode(array(1, 2, 3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "stack(2, explode(array(1, 2, 3)))" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out new file mode 100644 index 000000000000..d3735acf0f08 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out @@ -0,0 +1,689 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query schema +struct +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) +-- !query schema +struct<(udf((a + 1)) + 1):int,udf(count(b)):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "udf(COUNT(b))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 58, + "fragment" : "k" + } ] +} + + +-- !query +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT udf(1) FROM range(10) HAVING true +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT udf(id) FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true + + +-- !query +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query schema +struct +-- !query output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k) +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(udf(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(udf(1))" + } ] +} + + +-- !query +SELECT some(udf(1S)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "some(udf(1S))" + } ] +} + + +-- !query +SELECT any(udf(1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "any(udf(1L))" + } ] +} + + +-- !query +SELECT udf(every("true")) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 24, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/array.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/array.sql new file mode 100644 index 000000000000..865dc8bac4ea --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/array.sql @@ -0,0 +1,183 @@ +-- test cases for array functions + +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c); + +select * from data; + +-- index into array +select a, b[0], b[0] + b[1] from data; + +-- index into array of arrays +select a, c[0][0] + c[0][0 + 1] from data; + + +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +); + +select * from primitive_arrays; + +-- array_contains on all primitive types: result should alternate between true and false +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays; + +-- array_contains on nested arrays +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data; + +-- sort_array +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays; + +-- sort_array with an invalid string literal for the argument of sort order. +select sort_array(array('b', 'd'), '1'); + +-- sort_array with an invalid null literal casted as boolean for the argument of sort order. +select sort_array(array('b', 'd'), cast(NULL as boolean)); + +-- size +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays; + +-- index out of range for array elements +select element_at(array(1, 2, 3), 5); +select element_at(array(1, 2, 3), -5); +select element_at(array(1, 2, 3), 0); + +select elt(4, '123', '456'); +select elt(0, '123', '456'); +select elt(-1, '123', '456'); +select elt(null, '123', '456'); +select elt(null, '123', null); +select elt(1, '123', null); +select elt(2, '123', null); + +select array(1, 2, 3)[5]; +select array(1, 2, 3)[-1]; + +-- array_size +select array_size(array()); +select array_size(array(true)); +select array_size(array(2, 1)); +select array_size(NULL); +select array_size(map('a', 1, 'b', 2)); + +-- size(arrays_zip) +select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))); + +-- isnotnull(arrays_zip) +select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10))); +select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10))); + +-- function get() +select get(array(1, 2, 3), 0); +select get(array(1, 2, 3), 3); +select get(array(1, 2, 3), null); +select get(array(1, 2, 3), -1); + +-- function array_insert() +select array_insert(array(1, 2, 3), 3, 4); +select array_insert(array(2, 3, 4), 0, 1); +select array_insert(array(2, 3, 4), 1, 1); +select array_insert(array(1, 3, 4), -2, 2); +select array_insert(array(1, 2, 3), 3, "4"); +select array_insert(cast(NULL as ARRAY), 1, 1); +select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4); +select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)); +select array_insert(array(2, 3, NULL, 4), 5, 5); +select array_insert(array(2, 3, NULL, 4), -5, 1); +select array_insert(array(1), 2, cast(2 as tinyint)); + +set spark.sql.legacy.negativeIndexInArrayInsert=true; +select array_insert(array(1, 3, 4), -2, 2); +select array_insert(array(2, 3, NULL, 4), -5, 1); +set spark.sql.legacy.negativeIndexInArrayInsert=false; + +-- function array_compact +select array_compact(id) from values (1) as t(id); +select array_compact(array("1", null, "2", null)); +select array_compact(array("a", "b", "c")); +select array_compact(array(1D, null, 2D, null)); +select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6))); +select array_compact(array(null)); + +-- function array_append +select array_append(array(1, 2, 3), 4); +select array_append(array('a', 'b', 'c'), 'd'); +select array_append(array(1, 2, 3, NULL), NULL); +select array_append(array('a', 'b', 'c', NULL), NULL); +select array_append(CAST(null AS ARRAY), 'a'); +select array_append(CAST(null AS ARRAY), CAST(null as String)); +select array_append(array(), 1); +select array_append(CAST(array() AS ARRAY), CAST(NULL AS String)); +select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)); + +-- function array_prepend +select array_prepend(array(1, 2, 3), 4); +select array_prepend(array('a', 'b', 'c'), 'd'); +select array_prepend(array(1, 2, 3, NULL), NULL); +select array_prepend(array('a', 'b', 'c', NULL), NULL); +select array_prepend(CAST(null AS ARRAY), 'a'); +select array_prepend(CAST(null AS ARRAY), CAST(null as String)); +select array_prepend(array(), 1); +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)); +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)); + +-- SPARK-45599: Confirm 0.0, -0.0, and NaN are handled appropriately. +select array_union(array(0.0, -0.0, DOUBLE("NaN")), array(0.0, -0.0, DOUBLE("NaN"))); +select array_distinct(array(0.0, -0.0, -0.0, DOUBLE("NaN"), DOUBLE("NaN"))); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql new file mode 100644 index 000000000000..f9dfd161d0c0 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql @@ -0,0 +1,77 @@ +-- test cases for bitwise functions + +-- null +select bit_count(null); + +-- boolean +select bit_count(true); +select bit_count(false); + +-- byte/tinyint +select bit_count(cast(1 as tinyint)); +select bit_count(cast(2 as tinyint)); +select bit_count(cast(3 as tinyint)); + +-- short/smallint +select bit_count(1S); +select bit_count(2S); +select bit_count(3S); + +-- int +select bit_count(1); +select bit_count(2); +select bit_count(3); + +-- long/bigint +select bit_count(1L); +select bit_count(2L); +select bit_count(3L); + +-- negative num +select bit_count(-1L); + +-- edge value +select bit_count(9223372036854775807L); +select bit_count(-9223372036854775808L); + +-- other illegal arguments +select bit_count("bit count"); +select bit_count('a'); + +-- test for bit_xor +-- +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); + +-- empty case +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null; + +-- the suffix numbers show the expected answer +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test; + +-- group by +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1; + +--having +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; + +-- getbit +select getbit(11L, 3), getbit(11L, 2), getbit(11L, 1), getbit(11L, 0); +select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(cast(11L / 1 AS long), 1 - 1); +select getbit(11L, 63); +select getbit(11L, -1); +select getbit(11L, 64); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql new file mode 100644 index 000000000000..3b8ffcff1899 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql @@ -0,0 +1,130 @@ +create table char_tbl(c char(5), v varchar(6)) using parquet; +desc formatted char_tbl; +desc formatted char_tbl c; +show create table char_tbl; + +create table char_tbl2 using parquet as select * from char_tbl; +show create table char_tbl2; +desc formatted char_tbl2; +desc formatted char_tbl2 c; + +create table char_tbl3 like char_tbl; +desc formatted char_tbl3; +desc formatted char_tbl3 c; +show create table char_tbl3; + +create view char_view as select * from char_tbl; +desc formatted char_view; +desc formatted char_view c; +show create table char_view; + +alter table char_tbl rename to char_tbl1; +desc formatted char_tbl1; + +alter table char_tbl1 change column c type char(6); +alter table char_tbl1 change column c type char(5); +desc formatted char_tbl1; + +alter table char_tbl1 add columns (d char(5)); +desc formatted char_tbl1; + +alter view char_view as select * from char_tbl2; +desc formatted char_view; + +alter table char_tbl1 SET TBLPROPERTIES('yes'='no'); +desc formatted char_tbl1; + +alter view char_view SET TBLPROPERTIES('yes'='no'); +desc formatted char_view; + +alter table char_tbl1 UNSET TBLPROPERTIES('yes'); +desc formatted char_tbl1; + +alter view char_view UNSET TBLPROPERTIES('yes'); +desc formatted char_view; + +alter table char_tbl1 SET SERDEPROPERTIES('yes'='no'); +desc formatted char_tbl1; + +create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2); +desc formatted char_part; + +alter table char_part add partition (v2='ke', c2='nt') location 'loc1'; +desc formatted char_part; + +alter table char_part partition (v2='ke') rename to partition (v2='nt'); +desc formatted char_part; + +alter table char_part partition (v2='ke', c2='nt') set location 'loc2'; +desc formatted char_part; + +MSCK REPAIR TABLE char_part; +desc formatted char_part; + +create temporary view str_view as select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v); + +create table char_tbl4(c7 char(7), c8 char(8), v varchar(6), s string) using parquet; +insert into char_tbl4 select c, c, v, c from str_view; + +select c7, c8, v, s from char_tbl4; +select c7, c8, v, s from char_tbl4 where c7 = c8; +select c7, c8, v, s from char_tbl4 where c7 = v; +select c7, c8, v, s from char_tbl4 where c7 = s; +select c7, c8, v, s from char_tbl4 where c7 = 'NetEase '; +select c7, c8, v, s from char_tbl4 where v = 'Spark '; +select c7, c8, v, s from char_tbl4 order by c7; +select c7, c8, v, s from char_tbl4 order by v; + +select ascii(c7), ascii(c8), ascii(v), ascii(s) from char_tbl4; +select base64(c7), base64(c8), base64(v), ascii(s) from char_tbl4; +select bit_length(c7), bit_length(c8), bit_length(v), bit_length(s) from char_tbl4; +select char_length(c7), char_length(c8), char_length(v), char_length(s) from char_tbl4; +select octet_length(c7), octet_length(c8), octet_length(v), octet_length(s) from char_tbl4; +select concat_ws('|', c7, c8), concat_ws('|', c7, v), concat_ws('|', c7, s), concat_ws('|', v, s) from char_tbl4; +select concat(c7, c8), concat(c7, v), concat(c7, s), concat(v, s) from char_tbl4; +select like(c7, 'Ne _'), like(c8, 'Ne _') from char_tbl4; +select like(v, 'Spark_') from char_tbl4; +select c7 = c8, upper(c7) = upper(c8), lower(c7) = lower(c8) from char_tbl4 where s = 'NetEase'; +select c7 = s, upper(c7) = upper(s), lower(c7) = lower(s) from char_tbl4 where s = 'NetEase'; +select c7 = 'NetEase', upper(c7) = upper('NetEase'), lower(c7) = lower('NetEase') from char_tbl4 where s = 'NetEase'; +select printf('Hey, %s%s%s%s', c7, c8, v, s) from char_tbl4; +select repeat(c7, 2), repeat(c8, 2), repeat(v, 2), repeat(s, 2) from char_tbl4; +select replace(c7, 'Net', 'Apache'), replace(c8, 'Net', 'Apache'), replace(v, 'Spark', 'Kyuubi'), replace(s, 'Net', 'Apache') from char_tbl4; +select rpad(c7, 10), rpad(c8, 5), rpad(v, 5), rpad(s, 5) from char_tbl4; +select rtrim(c7), rtrim(c8), rtrim(v), rtrim(s) from char_tbl4; +select split(c7, 'e'), split(c8, 'e'), split(v, 'a'), split(s, 'e') from char_tbl4; +select substring(c7, 2), substring(c8, 2), substring(v, 3), substring(s, 2) from char_tbl4; +select left(c7, 2), left(c8, 2), left(v, 3), left(s, 2) from char_tbl4; +select right(c7, 2), right(c8, 2), right(v, 3), right(s, 2) from char_tbl4; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +select typeof(c7), typeof(c8), typeof(v), typeof(s) from char_tbl4 limit 1; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +select cast(c7 as char(1)), cast(c8 as char(10)), cast(v as char(1)), cast(v as varchar(1)), cast(s as char(5)) from char_tbl4; + +-- char_tbl has renamed to char_tbl1 +drop table char_tbl1; +drop table char_tbl2; +drop table char_tbl3; +drop table char_tbl4; + +-- ascii value for Latin-1 Supplement characters +select ascii('§'), ascii('÷'), ascii('×10'); +select chr(167), chr(247), chr(215); + +-- to_varchar is an alias for to_char +SELECT to_varchar(78.12, '$99.99'); +SELECT to_varchar(111.11, '99.9'); +SELECT to_varchar(12454.8, '99,999.9S'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/count.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/count.sql new file mode 100644 index 000000000000..3e673f95819e --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/count.sql @@ -0,0 +1,57 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData; + +-- distinct count with single expression +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData; + +-- count with multiple expressions +SELECT count(a, b), count(b, a), count(testData.*, testData.*) FROM testData; + +-- distinct count with multiple expressions +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*, testData.*) +FROM testData; + +-- distinct count with multiple literals +SELECT count(DISTINCT 3,2); +SELECT count(DISTINCT 2), count(DISTINCT 2,3); +SELECT count(DISTINCT 2), count(DISTINCT 3,2); +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData; +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData; + +-- legacy behavior: allow calling function count without parameters +set spark.sql.legacy.allowParameterlessCount=true; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; + +SELECT count() FROM testData; + +-- count without expressions +set spark.sql.legacy.allowParameterlessCount=false; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT count() FROM testData; + +-- legacy behavior: allow count(testData.*) +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=true; +SELECT count(testData.*) FROM testData; + +-- count with a single tblName.* as parameter +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=false; +SELECT count(testData.*) FROM testData; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/cte.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/cte.sql new file mode 100644 index 000000000000..ca88bbd42f13 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/cte.sql @@ -0,0 +1,181 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); + +-- WITH clause should not fall into infinite loop by referencing self +WITH s AS (SELECT 1 FROM s) SELECT * FROM s; + +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r; + +-- WITH clause should reference the base table +WITH t AS (SELECT 1 FROM t) SELECT * FROM t; + +-- WITH clause should not allow cross reference +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2; + +-- WITH clause should reference the previous CTE +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2; + +-- SPARK-18609 CTE with self-join +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2; + +-- CTE with column alias +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1; + +-- CTE with multiple column aliases +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2; + +-- CTE with duplicate column aliases +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t; + +-- CTE with empty column alias list is not allowed +WITH t() AS (SELECT 1) +SELECT * FROM t; + +-- CTEs with duplicate names are not allowed +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t; + +-- invalid CTE relation should fail the query even if it's not referenced +WITH t AS (SELECT 1 FROM non_existing_table) +SELECT 2; + +-- The following tests are ported from Postgres +-- Multiple uses are evaluated only once +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT random() FROM range(1, 5)) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- Deeply nested +WITH w1(c1) AS + (WITH w2(c2) AS + (WITH w3(c3) AS + (WITH w4(c4) AS + (WITH w5(c5) AS + (WITH w6(c6) AS + (WITH w7(c7) AS + (WITH w8(c8) AS + (SELECT 1) + SELECT * FROM w8) + SELECT * FROM w7) + SELECT * FROM w6) + SELECT * FROM w5) + SELECT * FROM w4) + SELECT * FROM w3) + SELECT * FROM w2) +SELECT * FROM w1; + +-- CTE referencing an outer-level variable, should fail +SELECT ( WITH cte(foo) AS ( VALUES(id) ) + SELECT (SELECT foo FROM cte) ) +FROM t; + +-- CTE name collision with subquery name +WITH same_name AS (SELECT 42) +SELECT * FROM same_name, (SELECT 10) AS same_name; + +-- CTE name collision with subquery name, should fail +WITH same_name(x) AS (SELECT 42) +SELECT same_name.x FROM (SELECT 10) AS same_name(x), same_name; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- Test behavior with an unknown-type literal in the WITH +WITH q AS (SELECT 'foo' AS x) +SELECT x, typeof(x) FROM q; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- The following tests are ported from ZetaSQL +-- Alias inside the with hides the underlying column name, should fail +with cte as (select id as id_alias from t) +select id from cte; + +-- Reference of later WITH, should fail. +with r1 as (select * from r2), + r2 as (select 1) +select 2; + +-- WITH in a table subquery +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q); + +-- WITH in an expression subquery +select (with q as (select 1 x) select * from q); + +-- WITH in an IN subquery +select 1 in (with q as (select 1) select * from q); + +-- WITH alias referenced outside its scope, should fail +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q), + q; + +-- References to CTEs of the same name should be resolved properly +WITH T1 as (select 1 a) +select * +from + T1 x, + (WITH T1 as (select 2 b) select * from T1) y, + T1 z; + +-- References to CTEs of the same name should be resolved properly +WITH TTtt as (select 1 a), + `tTTt_2` as (select 2 a) +select * +from + (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`); + +-- Correlated CTE subquery +select + (WITH q AS (select T.x) select * from q) +from (select 1 x, 2 y) T; + +-- The main query inside WITH can be correlated. +select + (WITH q AS (select 3 z) select x + t.y + z from q) +from (select 1 x, 2 y) T; + +-- A WITH subquery alias is visible inside a WITH clause subquery. +WITH q1 as (select 1 x) +select * from + (with q2 as (select * from q1) select * from q2); + +-- A WITH subquery alias is visible inside a WITH clause subquery, and they have the same name. +WITH q1 as (select 1 x) +select * from + (with q1 as (select x+1 from q1) select * from q1); + +-- The following tests are ported from DuckDB +-- Duplicate CTE alias, should fail +with cte1 as (select 42), cte1 as (select 42) select * FROM cte1; + +-- Refer to CTE in subquery +with cte1 as (Select id as j from t) +select * from cte1 where j = (select max(j) from cte1 as cte2); + +-- Nested CTE views that re-use CTE aliases +with cte AS (SELECT * FROM va) SELECT * FROM cte; + +-- Self-refer to non-existent cte, should fail. +with cte as (select * from cte) select * from cte; + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/date.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/date.sql new file mode 100644 index 000000000000..c1ae432d6316 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/date.sql @@ -0,0 +1,168 @@ +-- date literals, functions and operations + +create temporary view date_view as select '2011-11-11' date_str, '1' int_str; + +select date '2019-01-01\t'; +select date '2020-01-01中文'; + +select make_date(2019, 1, 1), make_date(12, 12, 12); +-- invalid month +select make_date(2000, 13, 1); +-- invalid day +select make_date(2000, 1, 33); + +-- invalid: year field must have at least 4 digits +select date'015'; +-- invalid: month field can have at most 2 digits +select date'2021-4294967297-11'; + +select current_date = current_date; +-- under ANSI mode, `current_date` can't be a function name. +select current_date() = current_date(); +select curdate(1); + +-- conversions between date and unix_date (number of days from epoch) +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null); +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null); + +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd'); + +-- `dayofweek` accepts both date and timestamp ltz/ntz inputs. +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15'); + +-- `weekday` accepts both date and timestamp ltz/ntz inputs. +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15'); + +-- `year` accepts both date and timestamp ltz/ntz inputs. +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15'); + +-- `month` accepts both date and timestamp ltz/ntz inputs. +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15'); + +-- `dayOfYear` accepts both date and timestamp ltz/ntz inputs. +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15'); + +-- next_day +select next_day("2015-07-23", "Mon"); +select next_day("2015-07-23", "xx"); +select next_day("2015-07-23 12:12:12", "Mon"); +-- next_date does not accept timestamp lzt/ntz input +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon"); +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon"); +select next_day("xx", "Mon"); +select next_day(null, "Mon"); +select next_day(null, "xx"); + +-- date add +select date_add(date'2011-11-11', 1); +select date_add('2011-11-11', 1); +select date_add('2011-11-11', 1Y); +select date_add('2011-11-11', 1S); +-- invalid cases: the second parameter can only be byte/short/int +select date_add('2011-11-11', 1L); +select date_add('2011-11-11', 1.0); +select date_add('2011-11-11', 1E1); +-- the second parameter can be a string literal if it can be parsed to int +select date_add('2011-11-11', '1'); +select date_add('2011-11-11', '1.2'); +-- null input leads to null result. +select date_add(null, 1); +select date_add(date'2011-11-11', null); +-- `date_add` accepts both date and timestamp ltz/ntz inputs (non-ANSI mode). +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1); +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1); + +-- date sub +select date_sub(date'2011-11-11', 1); +select date_sub('2011-11-11', 1); +select date_sub('2011-11-11', 1Y); +select date_sub('2011-11-11', 1S); +-- invalid cases: the second parameter can only be byte/short/int +select date_sub('2011-11-11', 1L); +select date_sub('2011-11-11', 1.0); +select date_sub('2011-11-11', 1E1); +-- the second parameter can be a string literal if it can be parsed to int +select date_sub(date'2011-11-11', '1'); +select date_sub(date'2011-11-11', '1.2'); +-- null input leads to null result. +select date_sub(null, 1); +select date_sub(date'2011-11-11', null); +-- `date_sub` accepts both date and timestamp ltz/ntz inputs (non-ANSI mode). +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1); +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1); + +-- date add/sub with non-literal string column +select date_add('2011-11-11', int_str) from date_view; +select date_sub('2011-11-11', int_str) from date_view; +select date_add(date_str, 1) from date_view; +select date_sub(date_str, 1) from date_view; + +-- date +/- number +select date '2011-11-11' + 1E1; +select date '2001-09-28' + 7Y; +select 7S + date '2001-09-28'; +select date '2001-10-01' - 7; + +-- date - date +select date '2001-10-01' - date '2001-09-28'; +-- if one side is string/null literal, promote it to date type. +select date '2001-10-01' - '2001-09-28'; +select '2001-10-01' - date '2001-09-28'; +select date '2001-09-28' - null; +select null - date '2019-10-06'; +-- invalid: non-literal string column +select date_str - date '2001-09-28' from date_view; +select date '2001-09-28' - date_str from date_view; + +-- invalid: date + string literal +select date'2011-11-11' + '1'; +select '1' + date'2011-11-11'; + +-- null result: date + null +select date'2011-11-11' + null; +select null + date'2011-11-11'; + +-- date +/- interval and interval + date +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11'; + +-- Unsupported narrow text style +select to_date('26/October/2015', 'dd/MMMMM/yyyy'); +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')); + +-- Add a number of units to a timestamp or a date +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123'); +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456'); +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03'); +select date_add(MINUTE, -100, date'2022-02-25'); +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03'); +select date_add(DAY, 367, date'2022-02-25'); +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03'); +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03'); +select dateadd(QUARTER, 5, date'2022-02-25'); +select date_add(YEAR, 1, date'2022-02-25'); + +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123'); +select date_add('QUARTER', 5, date'2022-02-25'); + +-- Get the difference between timestamps or dates in the specified units +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001'); +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455'); +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01'); +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00'); +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03'); +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00'); +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03'); +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03'); +select datediff(QUARTER, date'2022-02-25', date'2023-05-25'); +select date_diff(YEAR, date'2022-02-25', date'2023-02-25'); + +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455'); +select datediff('YEAR', date'2022-02-25', date'2023-02-25'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql new file mode 100644 index 000000000000..22b29f3451ff --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql @@ -0,0 +1,3 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT date.sql +--IMPORT timestamp.sql diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql new file mode 100644 index 000000000000..286763d4cbc1 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql @@ -0,0 +1,30 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES --- + +-- parsing invalid value with pattern 'y' +-- disable since https://github.com/facebookincubator/velox/pull/12694 +-- select to_timestamp('294248', 'y'); -- out of year value range [0, 294247] + +-- disable since https://github.com/facebookincubator/velox/pull/12694 +-- select to_timestamp('1234567', 'yyyyyyy'); -- the length of 'y' pattern must be less than 7 + +-- in java 8 this case is invalid, but valid in java 11, disabled for jenkins +-- select to_timestamp('100', 'DD'); +-- The error message is changed since Java 11+ +-- select to_timestamp('366', 'DD'); +select to_timestamp('2019-366', 'yyyy-DDD'); +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD'); +-- add a special case to test csv, because the legacy formatter it uses is lenient then Spark should +-- throw SparkUpgradeException +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')); + +-- Datetime types parse error +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select cast("Unparseable" as timestamp); +select cast("Unparseable" as date); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql new file mode 100644 index 000000000000..ee1afe502ab7 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT datetime-parsing.sql diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql new file mode 100644 index 000000000000..b412e463fc3c --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql @@ -0,0 +1,29 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS --- + +-- parsing with pattern 'y'. +-- the range of valid year is [-290307, 294247], +-- but particularly, some thrift client use java.sql.Timestamp to parse timestamp, which allows +-- only positive year values less or equal than 9999. So the cases bellow only use [1, 9999] to pass +-- ThriftServerQueryTestSuite +select to_timestamp('1', 'y'); + +-- reduced two digit form is used, the range of valid year is 20-[01, 99] +select to_timestamp('00', 'yy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be in [3, 6] for 'yyy' +select to_timestamp('001', 'yyy'); + +-- the range of valid year is [-9999, 9999], the number of digits must be 4 for 'yyyy'. +select to_timestamp('0001', 'yyyy'); + +-- the range of valid year is [-99999, 99999], the number of digits must be 5 for 'yyyyy'. +select to_timestamp('00001', 'yyyyy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be 6 for 'yyyyyy'. +select to_timestamp('000001', 'yyyyyy'); + +-- parsing with pattern 'D' +select to_timestamp('2020-365', 'yyyy-DDD'); +select to_timestamp('2020-30-365', 'yyyy-dd-DDD'); +select to_timestamp('2020-12-350', 'yyyy-MM-DDD'); +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 000000000000..b773396c050d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,96 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- group by ordinal followed by order by +select a, count(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select count(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, count(b) from data group by k, 1; + +-- can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, 2); + +-- mixed cases: can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, b); + +-- can use ordinal with cube +select a, b, count(1) from data group by 1, 2 with cube; + +-- can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, 2); + +-- mixed cases: can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, b); + +-- can use ordinal with rollup +select a, b, count(1) from data group by 1, 2 with rollup; + +-- can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)); + +-- mixed cases: can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)); + +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)); + +-- range error +select a, b, count(1) from data group by a, -1; + +select a, b, count(1) from data group by a, 3; + +select a, b, count(1) from data group by cube(-1, 2); + +select a, b, count(1) from data group by cube(1, 3); + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql new file mode 100644 index 000000000000..ce1b422de319 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql @@ -0,0 +1,281 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- struct() in group by +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa); + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); + +-- SPARK-34581: Don't optimize out grouping expressions from aggregate expressions without aggregate function +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL; + +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL; + + +-- Histogram aggregates with different numeric input types +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col); + +-- SPARK-27974: Support ANSI Aggregate Function: array_agg +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col); +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a; + + +SELECT mode(a), mode(b) FROM testData; +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a; + + +-- SPARK-44846: PushFoldableIntoBranches in complex grouping expressions cause bindReference error +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c; + +-- SPARK-45599: Check that "weird" doubles group and sort as desired. +SELECT col1, count(*) AS cnt +FROM VALUES + (0.0), + (-0.0), + (double('NaN')), + (double('NaN')), + (double('Infinity')), + (double('Infinity')), + (-double('Infinity')), + (-double('Infinity')) +GROUP BY col1 +ORDER BY col1 +; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/hll.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/hll.sql new file mode 100644 index 000000000000..a0c29cb25a5b --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/hll.sql @@ -0,0 +1,76 @@ +-- Positive test cases +-- Create a table with some testing data. +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col); + +SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1; + +SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) +FROM VALUES (50), (60), (60), (60), (75), (100) tab(col); + +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES ('abc'), ('def'), ('abc'), ('ghi'), ('abc') tab(col); + +SELECT hll_sketch_estimate( + hll_union( + hll_sketch_agg(col1), + hll_sketch_agg(col2))) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) + FROM (SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)); + +-- Negative test cases +SELECT hll_sketch_agg(col) +FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col); + +SELECT hll_sketch_agg(col, 2) +FROM VALUES (50), (60), (60) tab(col); + +SELECT hll_sketch_agg(col, 40) +FROM VALUES (50), (60), (60) tab(col); + +SELECT hll_union( + hll_sketch_agg(col1, 12), + hll_sketch_agg(col2, 13)) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +SELECT hll_union_agg(sketch, false) +FROM (SELECT hll_sketch_agg(col, 12) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)); + +SELECT hll_union(1, 2) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +-- The HLL functions receive invalid buffers as inputs. +SELECT hll_sketch_estimate(CAST ('abc' AS BINARY)); + +SELECT hll_union(CAST ('abc' AS BINARY), CAST ('def' AS BINARY)); + +SELECT hll_union_agg(buffer, false) +FROM (SELECT CAST('abc' AS BINARY) AS buffer); + +-- Clean up +DROP TABLE IF EXISTS t1; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/interval.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/interval.sql new file mode 100644 index 000000000000..e4da28c2e758 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/interval.sql @@ -0,0 +1,387 @@ +-- test for intervals + +-- multiply and divide an interval by a number +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15'); +select interval 4 month 2 weeks 3 microseconds * 1.5; +select interval 2 years 4 months; +select interval 2 weeks 3 microseconds * 1.5; +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5; +select interval 2147483647 month * 2; +select interval 2147483647 month / 0.5; +select interval 2147483647 day * 2; +select interval 2147483647 day / 0.5; + +-- multiply and divide an interval by a string literal +select interval 2 second * '2'; +select interval 2 second / '2'; +select interval 2 year * '2'; +select interval 2 year / '2'; +select interval 2 second * 'a'; +select interval 2 second / 'a'; +select interval 2 year * 'a'; +select interval 2 year / 'a'; + +select '2' * interval 2 second; +select '2' * interval 2 year; +select 'a' * interval 2 second; +select 'a' * interval 2 year; + +-- invalid: string literal / interval +select '2' / interval 2 second; +select '2' / interval 2 year; + +-- interval operation with null and zero case +select interval '2 seconds' / 0; +select interval '2 seconds' / null; +select interval '2 seconds' * null; +select null * interval '2 seconds'; +select interval '2' year / 0; +select interval '2' year / null; +select interval '2' year * null; +select null * interval '2' year; + +-- invalid: divide by interval +select 2 / interval '2' year; +select 2 / interval '2' hour; +select null / interval '2' year; +select null / interval '2' hour; + +-- interval with a positive/negative sign +select -interval '-1 month 1 day -1 second'; +select -interval '-1 year 1 month'; +select -interval '-1 day 1 hour -1 minute 1 second'; +select -interval -1 month 1 day -1 second; +select -interval -1 year 1 month; +select -interval -1 day 1 hour -1 minute 1 second; +select +interval '-1 month 1 day -1 second'; +select +interval '-1 year 1 month'; +select +interval '-1 day 1 hour -1 minute 1 second'; +select +interval -1 month 1 day -1 second; +select +interval -1 year 1 month; +select +interval -1 day 1 hour -1 minute 1 second; +select interval -'1-1' year to month; +select interval -'-1-1' year to month; +select interval +'-1-1' year to month; +select interval - '1 2:3:4.001' day to second; +select interval +'1 2:3:4.001' day to second; +select interval -'-1 2:3:4.001' day to second; +select interval -'1' year; +select interval -'-1' year; +select interval -'11' month; +select interval -'-11' month; +select interval -'1' day; +select interval -'-1' day; +select interval -'23' hour; +select interval -'-23' hour; +select interval -'59' minute; +select interval -'-59' minute; +select interval -'59' second; +select interval -'-59' second; + +-- make intervals +select make_interval(1); +select make_interval(1, 2); +select make_interval(1, 2, 3); +select make_interval(1, 2, 3, 4); +select make_interval(1, 2, 3, 4, 5); +select make_interval(1, 2, 3, 4, 5, 6); +select make_interval(1, 2, 3, 4, 5, 6, 7.008009); +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456); +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789); + +-- make_dt_interval +select make_dt_interval(1); +select make_dt_interval(1, 2); +select make_dt_interval(1, 2, 3); +select make_dt_interval(1, 2, 3, 4.005006); +select make_dt_interval(1, 0, 0, 123456789012.123456); +select make_dt_interval(2147483647); + +-- make_ym_interval +select make_ym_interval(1); +select make_ym_interval(1, 2); +select make_ym_interval(0, 1); +select make_ym_interval(178956970, 7); +select make_ym_interval(178956970, 8); +select make_ym_interval(-178956970, -8); +select make_ym_interval(-178956970, -9); + +-- cast string to intervals +select cast('1 second' as interval); +select cast('+1 second' as interval); +select cast('-1 second' as interval); +select cast('+ 1 second' as interval); +select cast('- 1 second' as interval); +select cast('- -1 second' as interval); +select cast('- +1 second' as interval); + +-- interval literal +select interval 13.123456789 seconds, interval -13.123456789 second; +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond; +select interval 1 year 2 month; +select interval 4 day 5 hour 6 minute 7 seconds; +select interval 3 week 8 millisecond 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '30' year '25' month; +select interval '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0-0' year to month; +select interval '0 0:0:0' day to second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15' day to hour; +select interval '20 15:40' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40' hour to minute; +select interval '15:40:32.99899999' hour to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; +select interval 30 day day; +select interval 30 days days; + +-- invalid day-time string intervals +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '20 40:32.99899999' minute to second; + +-- ns is not supported +select interval 10 nanoseconds; + +-- map + interval test +select map(1, interval 1 day, 2, interval 3 week); +select map(1, interval 1 day, 2, interval 2 day); +select map(1, interval 1 year, 2, interval 2 month); +select map(1, interval 1 month, 2, interval 2 month); +select map(1, interval 1 week, 2, interval 2 day); +select map(1, interval 2 millisecond, 3, interval 3 microsecond); + +-- typed interval expression +select interval 'interval 3 year 1 month'; +select interval '3 year 1 month'; +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds'; +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds'; + +-- malformed interval literal +select interval 1 fake_unit; +select interval 1 year to month; +select interval '1' year to second; +select interval '10-9' year to month '2-1' year to month; +select interval '10-9' year to month '12:11:10' hour to second; +select interval '1 15:11' day to minute '12:11:10' hour to second; +select interval 1 year '2-1' year to month; +select interval 1 year '12:11:10' hour to second; +select interval '10-9' year to month '1' year; +select interval '12:11:10' hour to second '1' year; +select interval (-30) day; +select interval (a + 1) day; +select interval 30 day day day; +select interval (-30) days; +select interval (a + 1) days; +select interval 30 days days days; +SELECT INTERVAL '178956970-7' YEAR TO MONTH; +SELECT INTERVAL '178956970-8' YEAR TO MONTH; +SELECT INTERVAL '-178956970-8' YEAR TO MONTH; +SELECT INTERVAL -'178956970-8' YEAR TO MONTH; + +-- interval +/- interval +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day; +-- if one side is string/null literal, promote it to interval type. +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day; +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour; + +-- invalid: malformed interval string +select interval '2' year + '3-3'; +select interval '2' year - '4'; +select '4 11:11' - interval '4 22:12' day to minute; +select '4 12:12:12' + interval '4 22:12' day to minute; +-- invalid: non-literal string column +create temporary view interval_view as select '1' str; +select interval '2' year + str from interval_view; +select interval '2' year - str from interval_view; +select str - interval '4 22:12' day to minute from interval_view; +select str + interval '4 22:12' day to minute from interval_view; + +-- invalid: mixed year-month and day-time interval +select interval '2-2' year to month + interval '3' day; +select interval '3' day + interval '2-2' year to month; +select interval '2-2' year to month - interval '3' day; +select interval '3' day - interval '2-2' year to month; + +-- invalid: number +/- interval +select 1 - interval '2' second; +select 1 + interval '2' month; +select interval '2' second + 1; +select interval '2' month - 1; + +-- control characters as white spaces +select interval '\t interval 1 day'; +select interval 'interval \t 1\tday'; +select interval 'interval\t1\tday'; +select interval '1\t' day; +select interval '1 ' day; +select interval '2-2\t' year to month; +select interval '-\t2-2\t' year to month; +select interval '\n0 12:34:46.789\t' day to second; +select interval '\n-\t10\t 12:34:46.789\t' day to second; +select interval '中文 interval 1 day'; +select interval 'interval中文 1 day'; +select interval 'interval 1中文day'; + +-- interval overflow: if (ansi) exception else NULL +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); + +-- interval support for csv and json functions +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute'); +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month'); +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute'); +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month'); + +select interval '+'; +select interval '+.'; +select interval '1'; +select interval '1.2'; +select interval '- 2'; +select interval '1 day -'; +select interval '1 day 1'; + +select interval '1 day 2' day; +select interval 'interval 1' day; +select interval '-\t 1' day; + +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D; + +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D; + +SELECT INTERVAL '106751991 04' DAY TO HOUR; +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE; +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND; +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE; +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND; +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND; +SELECT INTERVAL '-106751991 04' DAY TO HOUR; +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE; +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND; +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE; +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND; +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND; + +SELECT INTERVAL '106751992 04' DAY TO HOUR; +SELECT INTERVAL '-106751992 04' DAY TO HOUR; +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE; +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE; +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND; +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND; + +SELECT INTERVAL '178956970' YEAR; +SELECT INTERVAL '-178956970' YEAR; +SELECT INTERVAL '2147483647' MONTH; +SELECT INTERVAL '-2147483647' MONTH; + +SELECT INTERVAL '106751991' DAY; +SELECT INTERVAL '-106751991' DAY; +SELECT INTERVAL '2562047788' HOUR; +SELECT INTERVAL '-2562047788' HOUR; +SELECT INTERVAL '153722867280' MINUTE; +SELECT INTERVAL '-153722867280' MINUTE; +SELECT INTERVAL '54.775807' SECOND; +SELECT INTERVAL '-54.775807' SECOND; + +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR; +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND; +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH; +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH; +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS; + +-- compare interval with string +SELECT INTERVAL '1' DAY < '1'; +SELECT INTERVAL '1' DAY = '1'; +SELECT INTERVAL '1' DAY > '1'; +SELECT '1' < INTERVAL '1' DAY; +SELECT '1' = INTERVAL '1' DAY; +SELECT '1' > INTERVAL '1' DAY; +SELECT INTERVAL '1' YEAR < '1'; +SELECT INTERVAL '1' YEAR = '1'; +SELECT INTERVAL '1' YEAR > '1'; +SELECT '1' < INTERVAL '1' YEAR; +SELECT '1' = INTERVAL '1' YEAR; +SELECT '1' > INTERVAL '1' YEAR; + +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH); +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE); +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS); +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH); +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE); +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS); +SELECT abs(INTERVAL '-10' YEAR); +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND); +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR); +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH); +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY); +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR); +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY); +SELECT signum(INTERVAL '-10' YEAR); +SELECT signum(INTERVAL '10' MONTH); +SELECT signum(INTERVAL '0-0' YEAR TO MONTH); +SELECT signum(INTERVAL '-10' DAY); +SELECT signum(INTERVAL '10' HOUR); +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND); +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10); +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10); +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10); +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql new file mode 100644 index 000000000000..9b9887b5e420 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql @@ -0,0 +1,52 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x); + +-- SPARK-37613: Support ANSI Aggregate Function: regr_count +SELECT regr_count(y, x) FROM testRegression; +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k; +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37613: Support ANSI Aggregate Function: regr_r2 +SELECT regr_r2(y, x) FROM testRegression; +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k; +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression; +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37672: Support ANSI Aggregate Function: regr_sxx +SELECT regr_sxx(y, x) FROM testRegression; +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37681: Support ANSI Aggregate Function: regr_sxy +SELECT regr_sxy(y, x) FROM testRegression; +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37702: Support ANSI Aggregate Function: regr_syy +SELECT regr_syy(y, x) FROM testRegression; +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-39230: Support ANSI Aggregate Function: regr_slope +SELECT regr_slope(y, x) FROM testRegression; +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37623: Support ANSI Aggregate Function: regr_intercept +-- SELECT regr_intercept(y, x) FROM testRegression; +-- SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +-- SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k; +-- SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql new file mode 100644 index 000000000000..b12ce86e9794 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,25 @@ +-- test for misc functions +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- Spark-32793: Rewrite AssertTrue with RaiseError +SELECT assert_true(true), assert_true(boolean(1)); +SELECT assert_true(false); +SELECT assert_true(boolean(0)); +SELECT assert_true(null); +SELECT assert_true(boolean(null)); +SELECT assert_true(false, 'custom error message'); + +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v); +SELECT raise_error('error message'); +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql new file mode 100644 index 000000000000..20da11018106 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql @@ -0,0 +1,389 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v); + +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k; + +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k; + +SELECT + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + k, + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +GROUP BY k +ORDER BY k; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2); + +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym), + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym), + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + median(dt), + percentile(dt, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt) +FROM intervals; + +SELECT + k, + median(ym), + percentile(ym, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY ym) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + median(dt2), + percentile(dt2, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2) +FROM intervals +GROUP BY k +ORDER BY k; + +-- SPARK-44871: Fix percentile_disc behaviour +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a); + +SET spark.sql.legacy.percentileDiscCalculation = true; + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a); + +SELECT + percentile_cont(b) WITHIN GROUP (ORDER BY a DESC) as p0 +FROM values (12, 0.25), (13, 0.25), (22, 0.25) as v(a, b); + +SET spark.sql.legacy.percentileDiscCalculation = false; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql new file mode 100644 index 000000000000..dac7670fcab0 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql @@ -0,0 +1,365 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- CREATE TABLE empsalary ( +-- depname string, +-- empno integer, +-- salary int, +-- enroll_date date +-- ) USING parquet; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- INSERT INTO empsalary VALUES ('develop', 10, 5200, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('sales', 1, 5000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('personnel', 5, 3500, '2007-12-10'); +-- INSERT INTO empsalary VALUES ('sales', 4, 4800, '2007-08-08'); +-- INSERT INTO empsalary VALUES ('personnel', 2, 3900, '2006-12-23'); +-- INSERT INTO empsalary VALUES ('develop', 7, 4200, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('develop', 9, 4500, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('sales', 3, 4800, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('develop', 8, 6000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('develop', 11, 5200, '2007-08-15'); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary; + +-- with GROUP BY +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname); + +-- [SPARK-28064] Order by does not accept a call to rank() +-- SELECT depname, empno, salary, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary) ORDER BY rank() OVER w; + +-- empty window specification +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10; + +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS (); + +-- no window operation +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten); + +-- cumulative aggregate +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10; + +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28065] ntile does not accept NULL as input +-- SELECT ntile(NULL) OVER (ORDER BY ten, four), ten, four FROM tenk1 LIMIT 2; + +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four, 0) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- last returns the last row of the frame, which is CURRENT ROW in ORDER BY window. +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten; + +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause +-- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two; + +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10; + +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10; + +-- opexpr with different windows evaluation. +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum; + +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- more than one window with GROUP BY +-- SELECT sum(salary), +-- row_number() OVER (ORDER BY depname), +-- sum(sum(salary)) OVER (ORDER BY depname DESC) +-- FROM empsalary GROUP BY depname; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- identical windows with different names +-- SELECT sum(salary) OVER w1, count(*) OVER w2 +-- FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary); + +-- subplan +-- Cannot specify window frame for lead function +-- SELECT lead(ten, (SELECT two FROM tenk1 WHERE s.unique2 = unique2)) OVER (PARTITION BY four ORDER BY ten) +-- FROM tenk1 s WHERE unique2 < 10; + +-- empty table +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- mixture of agg/wfunc in the same window +-- SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC); + +-- Cannot safely cast 'enroll_date': string to date; +-- SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( +-- SELECT *, +-- CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, +-- CASE WHEN +-- AVG(salary) OVER (PARTITION BY depname) < salary +-- THEN 200 END AS depadj FROM empsalary +-- )s; + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- window function over ungrouped agg over empty row set (bug before 9.1) +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42; + +-- window function with ORDER BY an expression involving aggregates (9.1 bug) +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten; + +-- window and aggregate with GROUP BY expression (9.2 bug) +-- explain +-- select first(max(x)) over (), y +-- from (select unique1 as x, ten+four as y from tenk1) ss +-- group by y; + +-- test non-default frame specifications +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause +-- SELECT first_value(unique1) over w, +-- nth_value(unique1, 2) over w AS nth_2, +-- last_value(unique1) over w, unique1, four +-- FROM tenk1 WHERE unique1 < 10 +-- WINDOW w AS (order by four range between current row and unbounded following); + +-- [SPARK-28501] Frame bound value must be a literal. +-- SELECT sum(unique1) over +-- (order by unique1 +-- rows (SELECT unique1 FROM tenk1 ORDER BY unique1 LIMIT 1) + 1 PRECEDING), +-- unique1 +-- FROM tenk1 WHERE unique1 < 10; + +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT * FROM v_window; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude current row) as sum_rows FROM range(1, 10) i; + +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude group) as sum_rows FROM range(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude ties) as sum_rows FROM generate_series(1, 10) i; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude no others) as sum_rows FROM generate_series(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i.id, sum(i.id) over (order by i.id groups between 1 preceding and 1 following) as sum_rows FROM range(1, 11) i; +-- SELECT * FROM v_window; + +DROP VIEW v_window; +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- DROP TABLE empsalary; +DROP VIEW tenk2; +DROP VIEW int4_tbl; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql new file mode 100644 index 000000000000..ab8aec2f640e --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql @@ -0,0 +1,463 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15'); + +-- Test in_range for other datetime datatypes + +-- Spark only supports timestamp +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet; + +-- Spark cannot safely cast string to timestamp +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54'); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp desc range between +-- '1 year' preceding and '1 year' following); + +-- RANGE offset PRECEDING/FOLLOWING error cases +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date, salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by depname range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between 1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between -1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between 1 preceding and -2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between '1 year' preceding and '-2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- GROUPS tests + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 following and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and 2 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 preceding), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 0 preceding and 0 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following +-- exclude current row), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude group), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude ties), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following),unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude current row), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude group), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude ties), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select first_value(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lead(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28508] Support for range frame+row frame in the same query +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select last(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lag(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude current row), +-- lead(salary) over(order by enroll_date groups between 1 following and 3 following exclude ties), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 following and 3 following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude group), +-- lag(salary) over(order by enroll_date groups between 1 following and 3 following exclude group), +-- salary, enroll_date from empsalary; + +-- Show differences in offset interpretation between ROWS, RANGE, and GROUPS +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- SELECT * FROM range(1, 36, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- select 1 union all select 1 union all select 1 union all +-- SELECT * FROM range(5, 50, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +-- with UNION +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0; + +-- check some degenerate cases +create table t1 (f1 int, f2 int) using parquet; +insert into t1 values (1,1),(1,2),(2,2); + +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2; -- error, must have order by + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1 order by f2 +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f1 order by f2 +-- groups between 2 preceding and 1 preceding) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f2 order by f2 +-- groups between 1 following and 2 following) +-- from t1 where f1 = f2; + +-- ordering by a non-integer constant is allowed +SELECT rank() OVER (ORDER BY length('abc')); + +-- can't order by another window function +-- [SPARK-28566] window functions should not be allowed in window definitions +-- SELECT rank() OVER (ORDER BY rank() OVER (ORDER BY random())); + +-- some other errors +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10; + +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10; + +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1; + +SELECT * FROM rank() OVER (ORDER BY random()); + +-- Original query: DELETE FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; + +-- Original query: DELETE FROM empsalary RETURNING rank() OVER (ORDER BY random()); +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()); + +-- [SPARK-28645] Throw an error on window redefinition +-- select count(*) OVER w FROM tenk1 WINDOW w AS (ORDER BY unique1), w AS (ORDER BY unique1); + +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1; + +-- [SPARK-28646] Allow usage of `count` only for parameterless aggregate function +-- select count() OVER () FROM tenk1; + +-- The output is the expected one: `range` is not a window or aggregate function. +SELECT range(1, 100) OVER () FROM empsalary; + +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1; + +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1; + +-- filter + +-- [SPARK-30182] Support nested aggregates +-- SELECT sum(salary), row_number() OVER (ORDER BY depname), sum( +-- sum(salary) FILTER (WHERE enroll_date > '2007-01-01') +-- ) +-- FROM empsalary GROUP BY depname; + +-- Test pushdown of quals into a subquery containing window functions + +-- pushdown is safe because all PARTITION BY clauses include depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname) depsalary, +-- min(salary) OVER (PARTITION BY depname || 'A', depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- pushdown is unsafe because there's a PARTITION BY clause without depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY enroll_date) enroll_salary, +-- min(salary) OVER (PARTITION BY depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node collapsing +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname order by empno) depsalary, +-- min(salary) OVER (PARTITION BY depname, empno order by enroll_date) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node reordering +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT +-- lead(1) OVER (PARTITION BY depname ORDER BY salary, enroll_date), +-- lag(1) OVER (PARTITION BY depname ORDER BY salary,enroll_date,empno) +-- FROM empsalary; + +-- cleanup +DROP TABLE empsalary; +DROP TABLE datetimes; +DROP TABLE t1; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql new file mode 100644 index 000000000000..2836f3d2a72f --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql @@ -0,0 +1,408 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Spark doesn't handle UDFs in SQL +-- test user-defined window function with named args and default args +-- CREATE FUNCTION nth_value_def(val anyelement, n integer = 1) RETURNS anyelement +-- LANGUAGE internal WINDOW IMMUTABLE STRICT AS 'window_nth_value'; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(n := 2, val := ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- +-- Test the basic moving-aggregate machinery +-- + +-- create aggregates that record the series of transform calls (these are +-- intentionally not true inverses) + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict (text) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- test strict and non-strict cases +-- SELECT +-- p::text || ',' || i::text || ':' || COALESCE(v::text, 'NULL') AS row, +-- logging_agg_nonstrict(v) over wnd as nstrict, +-- logging_agg_nonstrict_initcond(v) over wnd as nstrict_init, +-- logging_agg_strict(v::text) over wnd as strict, +-- logging_agg_strict_initcond(v) over wnd as strict_init +-- FROM (VALUES +-- (1, 1, NULL), +-- (1, 2, 'a'), +-- (1, 3, 'b'), +-- (1, 4, NULL), +-- (1, 5, NULL), +-- (1, 6, 'c'), +-- (2, 1, NULL), +-- (2, 2, 'x'), +-- (3, 1, 'z') +-- ) AS t(p, i, v) +-- WINDOW wnd AS (PARTITION BY P ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- and again, but with filter +-- SELECT +-- p::text || ',' || i::text || ':' || +-- CASE WHEN f THEN COALESCE(v::text, 'NULL') ELSE '-' END as row, +-- logging_agg_nonstrict(v) filter(where f) over wnd as nstrict_filt, +-- logging_agg_nonstrict_initcond(v) filter(where f) over wnd as nstrict_init_filt, +-- logging_agg_strict(v::text) filter(where f) over wnd as strict_filt, +-- logging_agg_strict_initcond(v) filter(where f) over wnd as strict_init_filt +-- FROM (VALUES +-- (1, 1, true, NULL), +-- (1, 2, false, 'a'), +-- (1, 3, true, 'b'), +-- (1, 4, false, NULL), +-- (1, 5, false, NULL), +-- (1, 6, false, 'c'), +-- (2, 1, false, NULL), +-- (2, 2, true, 'x'), +-- (3, 1, true, 'z') +-- ) AS t(p, i, f, v) +-- WINDOW wnd AS (PARTITION BY p ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- test that volatile arguments disable moving-aggregate mode +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) +-- over wnd as inverse, +-- logging_agg_strict(v::text || CASE WHEN random() < 0 then '?' ELSE '' END) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) filter(where true) +-- over wnd as inverse, +-- logging_agg_strict(v::text) filter(where random() >= 0) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that non-overlapping windows don't use inverse transitions +-- SELECT +-- logging_agg_strict(v::text) OVER wnd +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that returning NULL from the inverse transition functions +-- restarts the aggregation from scratch. The second aggregate is supposed +-- to test cases where only some aggregates restart, the third one checks +-- that one aggregate restarting doesn't cause others to restart. + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION sum_int_randrestart_minvfunc(int4, int4) RETURNS int4 AS +-- $$ SELECT CASE WHEN random() < 0.2 THEN NULL ELSE $1 - $2 END $$ +-- LANGUAGE SQL STRICT; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE sum_int_randomrestart (int4) +-- ( +-- stype = int4, +-- sfunc = int4pl, +-- mstype = int4, +-- msfunc = int4pl, +-- minvfunc = sum_int_randrestart_minvfunc +-- ); + +-- Spark doesn't handle UDFs in SQL +-- WITH +-- vs AS ( +-- SELECT i, (random() * 100)::int4 AS v +-- FROM generate_series(1, 100) AS i +-- ), +-- sum_following AS ( +-- SELECT i, SUM(v) OVER +-- (ORDER BY i DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS s +-- FROM vs +-- ) +-- SELECT DISTINCT +-- sum_following.s = sum_int_randomrestart(v) OVER fwd AS eq1, +-- -sum_following.s = sum_int_randomrestart(-v) OVER fwd AS eq2, +-- 100*3+(vs.i-1)*3 = length(logging_agg_nonstrict(''::text) OVER fwd) AS eq3 +-- FROM vs +-- JOIN sum_following ON sum_following.i = vs.i +-- WINDOW fwd AS ( +-- ORDER BY vs.i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING +-- ); + +-- +-- Test various built-in aggregates that have moving-aggregate support +-- + +-- test inverse transition functions handle NULLs properly +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,AVG(v::interval) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +-- The cast syntax is present in PgSQL for legacy reasons and Spark will not recognize a money field +-- SELECT i,SUM(v::money) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1.10'),(2,'2.20'),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,SUM(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v); + +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n); + +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +-- For the following queries Spark result differs from PgSQL: +-- Spark handles division by zero as 'NaN' instead of 'NULL', which is the PgSQL behaviour +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +-- test that inverse transition functions work with various frame options +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- [SPARK-29638] Spark handles 'NaN' as 0 in sums +-- ensure aggregate over numeric properly recovers from NaN values +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b); + +-- It might be tempting for someone to add an inverse trans function for +-- float and double precision. This should not be done as it can give incorrect +-- results. This test should fail if anyone ever does this without thinking too +-- hard about it. +-- [SPARK-28516] adds `to_char` +-- SELECT to_char(SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING),'999999999999999999999D9') +-- FROM (VALUES(1,1e20),(2,1)) n(i,n); + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- SELECT i, b, bool_and(b) OVER w, bool_or(b) OVER w +-- FROM (VALUES (1,true), (2,true), (3,false), (4,false), (5,true)) v(i,b) +-- WINDOW w AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING); + +-- Tests for problems with failure to walk or mutate expressions +-- within window frame clauses. + +-- [SPARK-37612] Support window frame ORDER BY i ROWS BETWEEN (('foo' < 'foobar')::integer) PRECEDING AND CURRENT ROW +-- test walker (fails with collation error if expressions are not walked) +-- SELECT array_agg(i) OVER w +-- FROM range(1,6) i +-- WINDOW w AS (ORDER BY i ROWS BETWEEN (('foo' < 'foobar')::integer) PRECEDING AND CURRENT ROW); + +-- Spark doesn't handle UDFs in SQL +-- test mutator (fails when inlined if expressions are not mutated) +-- CREATE FUNCTION pg_temp.f(group_size BIGINT) RETURNS SETOF integer[] +-- AS $$ +-- SELECT array_agg(s) OVER w +-- FROM generate_series(1,5) s +-- WINDOW w AS (ORDER BY s ROWS BETWEEN CURRENT ROW AND GROUP_SIZE FOLLOWING) +-- $$ LANGUAGE SQL STABLE; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/random.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/random.sql new file mode 100644 index 000000000000..a1aae7b8759d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/random.sql @@ -0,0 +1,17 @@ +-- rand with the seed 0 +SELECT rand(0); +SELECT rand(cast(3 / 7 AS int)); +SELECT rand(NULL); +SELECT rand(cast(NULL AS int)); + +-- rand unsupported data type +SELECT rand(1.0); + +-- randn with the seed 0 +SELECT randn(0L); +SELECT randn(cast(3 / 7 AS long)); +SELECT randn(NULL); +SELECT randn(cast(NULL AS long)); + +-- randn unsupported data type +SELECT rand('1') diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql new file mode 100644 index 000000000000..f9cc21ed1029 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql @@ -0,0 +1,79 @@ +-- regexp_extract +SELECT regexp_extract('1a 2b 14m', '\\d+'); +SELECT regexp_extract('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract('a b m', '(\\d+)?', 1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('1a 2b 14m', '(?l)'); + +-- regexp_extract_all +SELECT regexp_extract_all('1a 2b 14m', '\\d+'); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0); + +-- regexp_replace +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something'); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 1); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 2); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 8); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 26); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 27); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 30); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', null); + +-- regexp_like +SELECT regexp_like('1a 2b 14m', '\\d+b'); +SELECT regexp_like('1a 2b 14m', '[a-z]+b'); +SELECT regexp('1a 2b 14m', '\\d+b'); +SELECT regexp('1a 2b 14m', '[a-z]+b'); +SELECT rlike('1a 2b 14m', '\\d+b'); +SELECT rlike('1a 2b 14m', '[a-z]+b'); + +-- regexp_count +SELECT regexp_count('1a 2b 14m', '\\d+'); +SELECT regexp_count('1a 2b 14m', 'mmm'); +SELECT regexp_count('the fox', 'FOX'); +SELECT regexp_count('the fox', '(?i)FOX'); +SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+'); +SELECT regexp_count(null, 'abc'); +SELECT regexp_count('abc', null); + +-- regexp_substr +SELECT regexp_substr('1a 2b 14m', '\\d+'); +SELECT regexp_substr('1a 2b 14m', '\\d+ '); +SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)'); +SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)'); +SELECT regexp_substr('1a 2b 14m', ''); +SELECT regexp_substr('Spark', null); +SELECT regexp_substr(null, '.*'); + +-- regexp_instr +SELECT regexp_instr('abc', 'b'); +SELECT regexp_instr('abc', 'x'); +SELECT regexp_instr('ABC', '(?-i)b'); +SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)'); +SELECT regexp_instr('abc', null); +SELECT regexp_instr(null, 'b'); +SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql new file mode 100644 index 000000000000..12290c6da206 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql @@ -0,0 +1,255 @@ +-- Argument number exception +select concat_ws(); +select format_string(); + +-- A pipe operator for string concatenation +select 'a' || 'b' || 'c'; + +-- replace function +select replace('abc', 'b', '123'); +select replace('abc', 'b'); + +-- uuid +select length(uuid()), (uuid() <> uuid()); + +-- position +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null); + +-- left && right +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null); +select left(null, -2); +select left("abcd", -2), left("abcd", 0), left("abcd", 'a'); +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); +select right(null, -2); +select right("abcd", -2), right("abcd", 0), right("abcd", 'a'); + +-- split function +SELECT split('aa1cc2ee3', '[1-9]+'); +SELECT split('aa1cc2ee3', '[1-9]+', 2); +SELECT split('hello', ''); +SELECT split('', ''); +SELECT split('abc', null); +SELECT split(null, 'b'); + +-- split_part function +SELECT split_part('11.12.13', '.', 2); +SELECT split_part('11.12.13', '.', -1); +SELECT split_part('11.12.13', '.', -3); +SELECT split_part('11.12.13', '', 1); +SELECT split_part('11ab12ab13', 'ab', 1); +SELECT split_part('11.12.13', '.', 0); +SELECT split_part('11.12.13', '.', 4); +SELECT split_part('11.12.13', '.', 5); +SELECT split_part('11.12.13', '.', -5); +SELECT split_part(null, '.', 1); +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum); + +-- substring function +SELECT substr('Spark SQL', 5); +SELECT substr('Spark SQL', -3); +SELECT substr('Spark SQL', 5, 1); +SELECT substr('Spark SQL' from 5); +SELECT substr('Spark SQL' from -3); +SELECT substr('Spark SQL' from 5 for 1); +SELECT substring('Spark SQL', 5); +SELECT substring('Spark SQL', -3); +SELECT substring('Spark SQL', 5, 1); +SELECT substring('Spark SQL' from 5); +SELECT substring('Spark SQL' from -3); +SELECT substring('Spark SQL' from 5 for 1); + +-- trim +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz "); +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx'); +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx'); +SELECT trim(LEADING 'xyz' FROM 'zzzytest'); +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz'); +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD'); +SELECT trim(TRAILING 'xyz' FROM 'testxxzx'); +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx'); +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy'); + +-- btrim +SELECT btrim('xyxtrimyyx', 'xy'); +SELECT btrim(encode(" xyz ", 'utf-8')); +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')); +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')); + +-- Check lpad/rpad with invalid length parameter +SELECT lpad('hi', 'invalid_length'); +SELECT rpad('hi', 'invalid_length'); + +-- lpad for BINARY inputs +SELECT hex(lpad(unhex(''), 5)); +SELECT hex(lpad(unhex('aabb'), 5)); +SELECT hex(lpad(unhex('aabbcc'), 2)); +SELECT hex(lpad(unhex('123'), 2)); +SELECT hex(lpad(unhex('12345'), 2)); +SELECT hex(lpad(unhex(''), 5, unhex('1f'))); +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))); +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))); +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))); +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))); +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))); +SELECT hex(lpad(unhex(''), 6, unhex(''))); +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))); +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))); + +-- rpad for BINARY inputs +SELECT hex(rpad(unhex(''), 5)); +SELECT hex(rpad(unhex('aabb'), 5)); +SELECT hex(rpad(unhex('aabbcc'), 2)); +SELECT hex(rpad(unhex('123'), 2)); +SELECT hex(rpad(unhex('12345'), 2)); +SELECT hex(rpad(unhex(''), 5, unhex('1f'))); +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))); +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))); +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))); +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))); +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))); +SELECT hex(rpad(unhex(''), 6, unhex(''))); +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))); +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))); + +-- lpad/rpad with mixed STRING and BINARY input +SELECT lpad('abc', 5, x'57'); +SELECT lpad(x'57', 5, 'abc'); +SELECT rpad('abc', 5, x'57'); +SELECT rpad(x'57', 5, 'abc'); + +-- decode +select decode(); +select decode(encode('abc', 'utf-8')); +select decode(encode('abc', 'utf-8'), 'utf-8'); +select decode(1, 1, 'Southlake'); +select decode(2, 1, 'Southlake'); +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic'); +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic'); +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle'); +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks'); +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.'); + +-- contains +SELECT CONTAINS(null, 'Spark'); +SELECT CONTAINS('Spark SQL', null); +SELECT CONTAINS(null, null); +SELECT CONTAINS('Spark SQL', 'Spark'); +SELECT CONTAINS('Spark SQL', 'SQL'); +SELECT CONTAINS('Spark SQL', 'SPARK'); + +SELECT startswith('Spark SQL', 'ark'); +SELECT startswith('Spark SQL', 'Spa'); +SELECT startswith(null, 'Spark'); +SELECT startswith('Spark', null); +SELECT startswith(null, null); + +SELECT endswith('Spark SQL', 'QL'); +SELECT endswith('Spark SQL', 'Spa'); +SELECT endswith(null, 'Spark'); +SELECT endswith('Spark', null); +SELECT endswith(null, null); + +SELECT contains(x'537061726b2053514c', x'537061726b'); +SELECT contains(x'', x''); +SELECT contains(x'537061726b2053514c', null); +SELECT contains(12, '1'); +SELECT contains(true, 'ru'); +SELECT contains(x'12', 12); +SELECT contains(true, false); + +SELECT startswith(x'537061726b2053514c', x'537061726b'); +SELECT startswith(x'537061726b2053514c', x''); +SELECT startswith(x'', x''); +SELECT startswith(x'537061726b2053514c', null); + +SELECT endswith(x'537061726b2053514c', x'53516c'); +SELECT endsWith(x'537061726b2053514c', x'537061726b'); +SELECT endsWith(x'537061726b2053514c', x''); +SELECT endsWith(x'', x''); +SELECT endsWith(x'537061726b2053514c', null); + +-- to_number +select to_number('454', '000'); +select to_number('454.2', '000.0'); +select to_number('12,454', '00,000'); +select to_number('$78.12', '$00.00'); +select to_number('+454', 'S000'); +select to_number('-454', 'S000'); +select to_number('12,454.8-', '00,000.9MI'); +select to_number('00,454.8-', '00,000.9MI'); +select to_number('<00,454.8>', '00,000.9PR'); + +-- to_binary +-- base64 valid +select to_binary('', 'base64'); +select to_binary(' ', 'base64'); +select to_binary(' ab cd ', 'base64'); +select to_binary(' ab c=', 'base64'); +select to_binary(' ab cdef= = ', 'base64'); +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64'); +-- base64 invalid +select to_binary('a', 'base64'); +select to_binary('a?', 'base64'); +select to_binary('abcde', 'base64'); +select to_binary('abcd=', 'base64'); +select to_binary('a===', 'base64'); +select to_binary('ab==f', 'base64'); +-- utf-8 +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8'); +select to_binary('大千世界', 'utf8'); +select to_binary('', 'utf-8'); +select to_binary(' ', 'utf8'); +-- hex valid +select to_binary('737472696E67'); +select to_binary('737472696E67', 'hex'); +select to_binary(''); +select to_binary('1', 'hex'); +select to_binary('FF'); +select to_binary('123', 'hex'); +select to_binary('12345', 'hex'); +-- 'format' parameter can be any foldable string value, not just literal. +select to_binary('abc', concat('utf', '-8')); +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)); +-- 'format' parameter is case insensitive. +select to_binary('abc', 'Hex'); +-- null inputs lead to null result. +select to_binary('abc', null); +select to_binary(null, 'utf-8'); +select to_binary(null, null); +select to_binary(null, cast(null as string)); +-- invalid format +select to_binary('abc', 1); +select to_binary('abc', 'invalidFormat'); +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat'); +SELECT to_binary('abc', fmtField) FROM fmtTable; +-- Clean up +DROP VIEW IF EXISTS fmtTable; +-- luhn_check +-- basic cases +select luhn_check('4111111111111111'); +select luhn_check('5500000000000004'); +select luhn_check('340000000000009'); +select luhn_check('6011000000000004'); +select luhn_check('6011000000000005'); +select luhn_check('378282246310006'); +select luhn_check('0'); +-- spaces in the beginning/middle/end +select luhn_check('4111111111111111 '); +select luhn_check('4111111 111111111'); +select luhn_check(' 4111111111111111'); +-- space +select luhn_check(''); +select luhn_check(' '); +-- non-digits +select luhn_check('510B105105105106'); +select luhn_check('ABCDED'); +-- null +select luhn_check(null); +-- non string (test implicit cast) +select luhn_check(6011111111111117); +select luhn_check(6011111111111118); +select luhn_check(123.456); + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql new file mode 100644 index 000000000000..5c69b895c16e --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql @@ -0,0 +1,62 @@ +create temp view v (c) as values (1), (null); +create temp view v_empty (e) as select 1 where false; + +-- Note: tables and temp views hit different optimization/execution codepaths: expressions over temp views are evaled at query compilation time by ConvertToLocalRelation +create table t(c int) using json; +insert into t values (1), (null); +create table t2(d int) using json; +insert into t2 values (2); +create table t_empty(e int) using json; + + + +set spark.sql.legacy.nullInEmptyListBehavior = false; + +-- null IN (empty subquery) +-- Correct results: c in (emptylist) should always be false + +select c, c in (select e from t_empty) from t; +select c, c in (select e from v_empty) from v; +select c, c not in (select e from t_empty) from t; +select c, c not in (select e from v_empty) from v; + +-- constant null IN (empty subquery) - rewritten by NullPropagation rule +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; + +select null in (select e from t_empty); +select null in (select e from v_empty); +select null not in (select e from t_empty); +select null not in (select e from v_empty); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- IN subquery which is not rewritten to join - here we use IN in the ON condition because that is a case that doesn't get rewritten to join in RewritePredicateSubquery, so we can observe the execution behavior of InSubquery directly +-- Correct results: column t2.d should be NULL because the ON condition is always false +select * from t left join t2 on (t.c in (select e from t_empty)) is null; +select * from t left join t2 on (t.c not in (select e from t_empty)) is null; + + + +-- Test legacy behavior flag +set spark.sql.legacy.nullInEmptyListBehavior = true; + +-- constant null IN (empty subquery) - rewritten by NullPropagation rule +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; + +select null in (select e from t_empty); +select null in (select e from v_empty); +select null not in (select e from t_empty); +select null not in (select e from v_empty); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- IN subquery which is not rewritten to join - here we use IN in the ON condition because that is a case that doesn't get rewritten to join in RewritePredicateSubquery, so we can observe the execution behavior of InSubquery directly +-- Correct results: column t2.d should be NULL because the ON condition is always false +select * from t left join t2 on (t.c in (select e from t_empty)) is null; +select * from t left join t2 on (t.c not in (select e from t_empty)) is null; + +reset spark.sql.legacy.nullInEmptyListBehavior; + +drop table t; +drop table t2; +drop table t_empty; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 000000000000..79d427bc2099 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,126 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call with invalid number of arguments +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); + +-- range call with incompatible type +select * from range(array(1, 2, 3)); + +-- range call with illegal step +select * from range(0, 5, 0); + +-- range call with a mixed-case function name +select * from RaNgE(2); + +-- range call with alias +select i from range(0, 2) t(i); + +-- explode +select * from explode(array(1, 2)); +select * from explode(map('a', 1, 'b', 2)); + +-- explode with empty values +select * from explode(array()); +select * from explode(map()); + +-- explode with column aliases +select * from explode(array(1, 2)) t(c1); +select * from explode(map('a', 1, 'b', 2)) t(k, v); + +-- explode with non-deterministic values +select * from explode(array(rand(0))); + +-- explode with erroneous input +select * from explode(null); +select * from explode(null) t(c1); +select * from explode(1); +select * from explode(1, 2); +select * from explode(explode(array(1))); +select * from explode(array(1, 2)) t(c1, c2); + +-- explode_outer +select * from explode_outer(array(1, 2)); +select * from explode_outer(map('a', 1, 'b', 2)); +select * from explode_outer(array()); +select * from explode_outer(map()); + +-- table-valued functions with join +select * from range(2) join explode(array(1, 2)); +select * from range(2) join explode_outer(array()); + +-- inline +select * from inline(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y); +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- inline with erroneous input +select * from inline(null); +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c); + +-- inline_outer +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- posexplode +select * from posexplode(array()); +select * from posexplode(array(1, 2)); +select * from posexplode(array(1, 2)) t(pos, x); +select * from posexplode(map()); +select * from posexplode(map('a', 1, 'b', 2)); +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v); + +-- posexplode with erroneous input +select * from posexplode(1); +select * from posexplode(1, 2); +select * from posexplode(explode(array(1))); +select * from posexplode(array(1, 2)) t(x); + +-- posexplode +select * from posexplode_outer(array()); +select * from posexplode_outer(array(1, 2)); +select * from posexplode_outer(map()); +select * from posexplode_outer(map('a', 1, 'b', 2)); + +-- json_tuple +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y); +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b'); + +-- json_tuple with erroneous input +select * from json_tuple(); +select * from json_tuple('{"a": 1}'); +select * from json_tuple('{"a": 1}', 1); +select * from json_tuple('{"a": 1}', null); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x); + +-- stack +select * from stack(1, 1, 2, 3); +select * from stack(2, 1, 2, 3); +select * from stack(3, 1, 2, 3) t(x); +select * from stack(4, 1, 2, 3) t(x); +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c); +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c); + +-- stack with erroneous input +select * from stack(); +select * from stack(2, 1, 2, 3) t(a, b, c); +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b'); +select * from stack(2, explode(array(1, 2, 3))); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql new file mode 100644 index 000000000000..ecdbe65f3eb1 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql @@ -0,0 +1,145 @@ +-- timestamp literals, functions and operations + +select timestamp '2019-01-01\t'; +select timestamp '2019-01-01中文'; + +-- invalid: year too large +select timestamp'4294967297'; +-- invalid: minute field can have at most 2 digits +select timestamp'2021-01-01T12:30:4294967297.123456'; + +select current_timestamp = current_timestamp; +-- under ANSI mode, `current_timestamp` can't be a function name. +select current_timestamp() = current_timestamp(); + +select localtimestamp() = localtimestamp(); + +-- timestamp numeric fields constructor +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678); +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET'); +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007); +SELECT make_timestamp(1, 1, 1, 1, 1, 1); +SELECT make_timestamp(1, 1, 1, 1, 1, 60); +SELECT make_timestamp(1, 1, 1, 1, 1, 61); +SELECT make_timestamp(1, 1, 1, 1, 1, null); +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999); +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999); +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999); + +-- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS that always create timestamp_ltz +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null); +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)); +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null); +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null); +-- overflow exception +select TIMESTAMP_SECONDS(1230219000123123); +select TIMESTAMP_SECONDS(-1230219000123123); +select TIMESTAMP_MILLIS(92233720368547758); +select TIMESTAMP_MILLIS(-92233720368547758); +-- truncate exception +select TIMESTAMP_SECONDS(0.1234567); +-- truncation is OK for float/double +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)); + +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b); +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2; +select a, b from ttf2 order by a, current_date; + + +-- UNIX_SECONDS, UNIX_MILLISECONDS and UNIX_MICROSECONDS +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null); +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null); +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null); + +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd'); +select to_timestamp(1); +-- variable-length second fraction tests +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- second fraction exceeded max variable length +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- special cases +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); + +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'"); +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd"); + +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''"); -- tail +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss"); -- head +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss"); -- head but as single quote + +-- missing fields +select to_timestamp("2019 40", "yyyy mm"); +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss"); + +-- timestamp - timestamp +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'; +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; +-- if one side is string/null literal, promote it to timestamp type. +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'; +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'; +select timestamp'2011-11-11 11:11:11' - null; +select null - timestamp'2011-11-11 11:11:11'; +-- invalid: non-literal string column +create temporary view ts_view as select '2011-11-11 11:11:11' str; +select str - timestamp'2011-11-11 11:11:11' from ts_view; +select timestamp'2011-11-11 11:11:11' - str from ts_view; + +-- invalid: timestamp + string literal +select timestamp'2011-11-11 11:11:11' + '1'; +select '1' + timestamp'2011-11-11 11:11:11'; + +-- null result: timestamp + null +select timestamp'2011-11-11 11:11:11' + null; +select null + timestamp'2011-11-11 11:11:11'; + +-- timestamp +/- interval and interval + timestamp +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'; +-- promote date to timestamp if the interval is hour to second. +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01'; + +-- Unsupported narrow text style +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG'); +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); + +-- Add a number of units to a timestamp or a date +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03'); +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03'); +select timestampadd(YEAR, 1, date'2022-02-15'); +select timestampadd(SECOND, -1, date'2022-02-15'); + +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03'); +select timestampadd('SECOND', -1, date'2022-02-15'); + +-- Get the difference between timestamps in the specified units +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03'); +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03'); +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15'); +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59'); + +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03'); +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql new file mode 100644 index 000000000000..c7957e0f392b --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql @@ -0,0 +1,88 @@ +-- Numeric + Numeric +SELECT try_add(1, 1); +SELECT try_add(2147483647, 1); +SELECT try_add(2147483647, decimal(1)); +SELECT try_add(2147483647, "1"); +SELECT try_add(-2147483648, -1); +SELECT try_add(9223372036854775807L, 1); +SELECT try_add(-9223372036854775808L, -1); +SELECT try_add(1, 1.0 / 0.0); + +-- Date + Integer +SELECT try_add(date'2021-01-01', 1); +SELECT try_add(1, date'2021-01-01'); + +-- Date + Interval +SELECT try_add(date'2021-01-01', interval 2 year); +SELECT try_add(date'2021-01-01', interval 2 second); +SELECT try_add(interval 2 year, date'2021-01-01'); +SELECT try_add(interval 2 second, date'2021-01-01'); + +-- Timestamp + Interval +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year); +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second); +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00'); +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00'); + +-- Interval + Interval +SELECT try_add(interval 2 year, interval 2 year); +SELECT try_add(interval 2 second, interval 2 second); +SELECT try_add(interval 2 year, interval 2 second); +SELECT try_add(interval 2147483647 month, interval 2 month); +SELECT try_add(interval 106751991 day, interval 3 day); + +-- Numeric / Numeric +SELECT try_divide(1, 0.5); +SELECT try_divide(1, 0); +SELECT try_divide(0, 0); +SELECT try_divide(1, (2147483647 + 1)); +SELECT try_divide(1L, (9223372036854775807L + 1L)); +SELECT try_divide(1, 1.0 / 0.0); +SELECT try_divide(1, decimal(0)); +SELECT try_divide(1, "0"); + +-- Interval / Numeric +SELECT try_divide(interval 2 year, 2); +SELECT try_divide(interval 2 second, 2); +SELECT try_divide(interval 2 year, 0); +SELECT try_divide(interval 2 second, 0); +SELECT try_divide(interval 2147483647 month, 0.5); +SELECT try_divide(interval 106751991 day, 0.5); + +-- Numeric - Numeric +SELECT try_subtract(1, 1); +SELECT try_subtract(2147483647, -1); +SELECT try_subtract(2147483647, decimal(-1)); +SELECT try_subtract(2147483647, "-1"); +SELECT try_subtract(-2147483648, 1); +SELECT try_subtract(9223372036854775807L, -1); +SELECT try_subtract(-9223372036854775808L, 1); +SELECT try_subtract(1, (2147483647 + 1)); +SELECT try_subtract(1L, (9223372036854775807L + 1L)); +SELECT try_subtract(1, 1.0 / 0.0); + +-- Interval - Interval +SELECT try_subtract(interval 2 year, interval 3 year); +SELECT try_subtract(interval 3 second, interval 2 second); +SELECT try_subtract(interval 2147483647 month, interval -2 month); +SELECT try_subtract(interval 106751991 day, interval -3 day); + +-- Numeric * Numeric +SELECT try_multiply(2, 3); +SELECT try_multiply(2147483647, -2); +SELECT try_multiply(2147483647, decimal(-2)); +SELECT try_multiply(2147483647, "-2"); +SELECT try_multiply(-2147483648, 2); +SELECT try_multiply(9223372036854775807L, 2); +SELECT try_multiply(-9223372036854775808L, -2); +SELECT try_multiply(1, (2147483647 + 1)); +SELECT try_multiply(1L, (9223372036854775807L + 1L)); +SELECT try_multiply(1, 1.0 / 0.0); + +-- Interval * Numeric +SELECT try_multiply(interval 2 year, 2); +SELECT try_multiply(interval 2 second, 2); +SELECT try_multiply(interval 2 year, 0); +SELECT try_multiply(interval 2 second, 0); +SELECT try_multiply(interval 2147483647 month, 2); +SELECT try_multiply(interval 106751991 day, 2); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql new file mode 100644 index 000000000000..92c71e2bd4f6 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql @@ -0,0 +1,50 @@ +-- TRY_CAST string representing a valid fractional number to integral should truncate the number + +-- TRY_CAST string which are not numbers to integral should return null +SELECT TRY_CAST('abc' AS int); +SELECT TRY_CAST('abc' AS long); + +-- TRY_CAST empty string to integral should return null +SELECT TRY_CAST('' AS int); +SELECT TRY_CAST('' AS long); + +-- TRY_CAST null to integral should return null +SELECT TRY_CAST(NULL AS int); +SELECT TRY_CAST(NULL AS long); + +-- TRY_CAST invalid decimal string to integral should return null +SELECT TRY_CAST('123.a' AS int); +SELECT TRY_CAST('123.a' AS long); + +-- '-2147483648' is the smallest int value +SELECT TRY_CAST('-2147483648' AS int); +SELECT TRY_CAST('-2147483649' AS int); + +-- '2147483647' is the largest int value +SELECT TRY_CAST('2147483647' AS int); +SELECT TRY_CAST('2147483648' AS int); + +-- '-9223372036854775808' is the smallest long value +SELECT TRY_CAST('-9223372036854775808' AS long); +SELECT TRY_CAST('-9223372036854775809' AS long); + +-- '9223372036854775807' is the largest long value +SELECT TRY_CAST('9223372036854775807' AS long); +SELECT TRY_CAST('9223372036854775808' AS long); + +-- TRY_CAST string to interval and interval to string +SELECT TRY_CAST('interval 3 month 1 hour' AS interval); +SELECT TRY_CAST('abc' AS interval); + +-- TRY_CAST string to boolean +select TRY_CAST('true' as boolean); +select TRY_CAST('false' as boolean); +select TRY_CAST('abc' as boolean); + +-- TRY_CAST string to date +SELECT TRY_CAST("2021-01-01" AS date); +SELECT TRY_CAST("2021-101-01" AS date); + +-- TRY_CAST string to timestamp +SELECT TRY_CAST("2021-01-01 00:00:00" AS timestamp); +SELECT TRY_CAST("2021-101-01 00:00:00" AS timestamp); \ No newline at end of file diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql new file mode 100644 index 000000000000..755d0542f8a1 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql @@ -0,0 +1,4 @@ +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd'); +select try_to_timestamp(1); +select try_to_timestamp('2016-12-31 abc'); +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql new file mode 100644 index 000000000000..c02c2dcb34fd --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql @@ -0,0 +1,11 @@ +-- array input +SELECT try_element_at(array(1, 2, 3), 0); +SELECT try_element_at(array(1, 2, 3), 1); +SELECT try_element_at(array(1, 2, 3), 3); +SELECT try_element_at(array(1, 2, 3), 4); +SELECT try_element_at(array(1, 2, 3), -1); +SELECT try_element_at(array(1, 2, 3), -4); + +-- map input +SELECT try_element_at(map('a','b'), 'a'); +SELECT try_element_at(map('a','b'), 'abc'); diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql new file mode 100644 index 000000000000..731dbc2dcf31 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql @@ -0,0 +1,54 @@ +-- +-- 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. +-- + +CREATE TEMPORARY VIEW t AS SELECT 'aa' as a; + +-- casting to data types which are unable to represent the string input returns NULL +select cast(a as byte) from t; +select cast(a as short) from t; +select cast(a as int) from t; +select cast(a as long) from t; +select cast(a as float) from t; +select cast(a as double) from t; +select cast(a as decimal) from t; +select cast(a as boolean) from t; +select cast(a as timestamp) from t; +select cast(a as date) from t; +-- casting to binary works correctly +select cast(a as binary) from t; +-- casting to array, struct or map throws exception +select cast(a as array) from t; +select cast(a as struct) from t; +select cast(a as map) from t; + +-- all timestamp/date expressions return NULL if bad input strings are provided +select to_timestamp(a) from t; +select to_unix_timestamp(a) from t; +select unix_timestamp(a) from t; +select from_unixtime(a) from t; +select from_unixtime('2018-01-01', a) from t; +select next_day(a, 'MO') from t; +select next_day('2018-01-01', a) from t; +select trunc(a, 'MM') from t; +select trunc('2018-01-01', a) from t; + +-- some functions return NULL if bad input is provided +select unhex('-123'); +select sha2(a, a) from t; +select get_json_object(a, a) from t; +select json_tuple(a, a) from t; +select from_json(a, 'a INT') from t; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql new file mode 100644 index 000000000000..0cc57c97b020 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql @@ -0,0 +1,156 @@ +-- This test file was converted from group-by.sql. +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT udf(a), udf(COUNT(b)) FROM testData; +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a; +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b; +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a); + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate with complex GroupBy expressions. +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b; +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1; +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); + +-- Aggregate with nulls. +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a); + +-- Aliases in SELECT could be used in GROUP BY +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k; +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1; + +-- Aggregate functions cannot be used in GROUP BY +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a); + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a); + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT udf(COUNT(1)) FROM testData WHERE false; +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT udf(1) FROM range(10) HAVING true; + +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0; + +SELECT udf(id) FROM range(10) HAVING id > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k); + +-- having +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k); + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)); + +-- input type checking Int +SELECT every(udf(1)); + +-- input type checking Short +SELECT some(udf(1S)); + +-- input type checking Long +SELECT any(udf(1L)); + +-- input type checking String +SELECT udf(every("true")); + +-- every/some/any aggregates are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L; +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L; +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql new file mode 100644 index 000000000000..be69e5ffb879 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql @@ -0,0 +1,20 @@ +-- parse_url function +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO'); + +-- url_encode function +select url_encode('https://spark.apache.org'); +select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2'); +select url_encode(null); + +-- url_decode function +select url_decode('https%3A%2F%2Fspark.apache.org'); +select url_decode('http%3A%2F%2spark.apache.org'); +select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2'); +select url_decode(null); \ No newline at end of file diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/window.sql b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/window.sql new file mode 100644 index 000000000000..a1728a196170 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/inputs/window.sql @@ -0,0 +1,477 @@ +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM2 spark.sql.optimizer.windowGroupLimitThreshold=-1 +--CONFIG_DIM2 spark.sql.optimizer.windowGroupLimitThreshold=1000 + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate); + +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + +CREATE OR REPLACE TEMPORARY VIEW test_ignore_null AS SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v); + +-- RowsBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; + +-- RangeBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double; +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_date; + +-- RangeBetween with reverse OrderBy +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; + +-- Invalid window frame +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val; + + +-- Window functions +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +any_value(val) OVER w AS any_value, +any_value(val, true) OVER w AS any_value_ignore_null, +any_value(val, false) OVER w AS any_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct, +covar_pop(val, val_long) OVER w AS covar_pop, +corr(val, val_long) OVER w AS corr, +stddev_samp(val) OVER w AS stddev_samp, +stddev_pop(val) OVER w AS stddev_pop, +collect_list(val) OVER w AS collect_list, +collect_set(val) OVER w AS collect_set, +skewness(val_double) OVER w AS skewness, +kurtosis(val_double) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val; + +-- Null inputs +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val; + +-- OrderBy not specified +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val; + +-- Over clause is empty +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val; + +-- first_value()/last_value()/any_value() over () +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val; + +-- parentheses around window reference +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val); + +-- with filter predicate +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val; + +-- nth_value()/first_value()/any_value() over () +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + department, + salary, + FIRST_VALUE(employee_name) OVER w highest_salary, + NTH_VALUE(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +) +ORDER BY department; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC; + +SELECT + content, + id, + v, + lead(v, 0) IGNORE NULLS OVER w lead_0, + lead(v, 1) IGNORE NULLS OVER w lead_1, + lead(v, 2) IGNORE NULLS OVER w lead_2, + lead(v, 3) IGNORE NULLS OVER w lead_3, + lag(v, 0) IGNORE NULLS OVER w lag_0, + lag(v, 1) IGNORE NULLS OVER w lag_1, + lag(v, 2) IGNORE NULLS OVER w lag_2, + lag(v, 3) IGNORE NULLS OVER w lag_3, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY id; + +SELECT + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays; + +SELECT + SUM(salary) OVER w sum_salary +FROM + basic_pays; + +-- Test cases for InferWindowGroupLimit +create or replace temp view t1 (p, o) as values (1, 1), (1, 1), (1, 2), (2, 1), (2, 1), (2, 2); +select * from (select *, dense_rank() over (partition by p order by o) as rnk from t1) where rnk = 1; + +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/array.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/array.sql.out new file mode 100644 index 000000000000..ef1e96b1ba1d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/array.sql.out @@ -0,0 +1,794 @@ +-- Replace exception with GlutenException +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from data +-- !query schema +struct,c:array>> +-- !query output +one [11,12,13] [[111,112,113],[121,122,123]] +two [21,22,23] [[211,212,213],[221,222,223]] + + +-- !query +select a, b[0], b[0] + b[1] from data +-- !query schema +struct +-- !query output +one 11 23 +two 21 43 + + +-- !query +select a, c[0][0] + c[0][0 + 1] from data +-- !query schema +struct +-- !query output +one 223 +two 423 + + +-- !query +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from primitive_arrays +-- !query schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> +-- !query output +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00,2016-11-12 20:54:00] + + +-- !query +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays +-- !query schema +struct +-- !query output +true false true false true false true false true false true false true false true false true false true false + + +-- !query +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data +-- !query schema +struct +-- !query output +false false +true true + + +-- !query +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays +-- !query schema +struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> +-- !query output +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00,2016-11-15 20:54:00] + + +-- !query +select sort_array(array('b', 'd'), '1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"sort_array(array(b, d), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "sort_array(array('b', 'd'), '1')" + } ] +} + + +-- !query +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(NULL AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "2", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"sort_array(array(b, d), CAST(NULL AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "sort_array(array('b', 'd'), cast(NULL as boolean))" + } ] +} + + +-- !query +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays +-- !query schema +struct +-- !query output +1 2 2 2 2 2 2 2 2 2 + + +-- !query +select element_at(array(1, 2, 3), 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select element_at(array(1, 2, 3), -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +SQL array indices start at 1 + + +-- !query +select elt(4, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(0, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(-1, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(null, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(null, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(1, '123', null) +-- !query schema +struct +-- !query output +123 + + +-- !query +select elt(2, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array(1, 2, 3)[5] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array(1, 2, 3)[-1] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_size(array()) +-- !query schema +struct +-- !query output +0 + + +-- !query +select array_size(array(true)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select array_size(array(2, 1)) +-- !query schema +struct +-- !query output +2 + + +-- !query +select array_size(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_size(map('a', 1, 'b', 2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(a, 1, b, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_size(map(a, 1, b, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "array_size(map('a', 1, 'b', 2))" + } ] +} + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +false + + +-- !query +select get(array(1, 2, 3), 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +select get(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3), 3, 4) +-- !query schema +struct> +-- !query output +[1,2,4,3] + + +-- !query +select array_insert(array(2, 3, 4), 0, 1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +(0 vs. 0) Array insert position should not be 0. + + +-- !query +select array_insert(array(2, 3, 4), 1, 1) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,3,2,4] + + +-- !query +select array_insert(array(1, 2, 3), 3, "4") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"ARRAY\"", + "functionName" : "`array_insert`", + "leftType" : "\"ARRAY\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"array_insert(array(1, 2, 3), 3, 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "array_insert(array(1, 2, 3), 3, \"4\")" + } ] +} + + +-- !query +select array_insert(cast(NULL as ARRAY), 1, 1) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_insert(array(2, 3, NULL, 4), 5, 5) +-- !query schema +struct> +-- !query output +[2,3,null,4,5] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,2,3,null,4] + + +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query schema +struct> +-- !query output +[1,2] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=true +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert true + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,null,2,3,null,4] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=false +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert false + + +-- !query +select array_compact(id) from values (1) as t(id) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"id\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_compact(id)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "array_compact(id)" + } ] +} + + +-- !query +select array_compact(array("1", null, "2", null)) +-- !query schema +struct> +-- !query output +["1","2"] + + +-- !query +select array_compact(array("a", "b", "c")) +-- !query schema +struct> +-- !query output +["a","b","c"] + + +-- !query +select array_compact(array(1D, null, 2D, null)) +-- !query schema +struct> +-- !query output +[1.0,2.0] + + +-- !query +select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6))) +-- !query schema +struct>> +-- !query output +[[1,2,3,null],[4,null,6]] + + +-- !query +select array_compact(array(null)) +-- !query schema +struct> +-- !query output +[] + + +-- !query +select array_append(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_append(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["a","b","c","d"] + + +-- !query +select array_append(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_append(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +["a","b","c",null,null] + + +-- !query +select array_append(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_append(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_prepend(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[4,1,2,3] + + +-- !query +select array_prepend(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["d","a","b","c"] + + +-- !query +select array_prepend(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[null,1,2,3,null] + + +-- !query +select array_prepend(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +[null,"a","b","c",null] + + +-- !query +select array_prepend(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_union(array(0.0, -0.0, DOUBLE("NaN")), array(0.0, -0.0, DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] + + +-- !query +select array_distinct(array(0.0, -0.0, -0.0, DOUBLE("NaN"), DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out new file mode 100644 index 000000000000..852e50f17b5d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out @@ -0,0 +1,306 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select bit_count(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select bit_count(true) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(false) +-- !query schema +struct +-- !query output +0 + + +-- !query +select bit_count(cast(1 as tinyint)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(cast(2 as tinyint)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(cast(3 as tinyint)) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1S) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2S) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3S) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3L) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(-1L) +-- !query schema +struct +-- !query output +64 + + +-- !query +select bit_count(9223372036854775807L) +-- !query schema +struct +-- !query output +63 + + +-- !query +select bit_count(-9223372036854775808L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count("bit count") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"bit count\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(bit count)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "bit_count(\"bit count\")" + } ] +} + + +-- !query +select bit_count('a') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "bit_count('a')" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0 +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test +-- !query schema +struct +-- !query output +4 5 2 2 2 + + +-- !query +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1 +-- !query schema +struct +-- !query output +4 +6 + + +-- !query +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query schema +struct +-- !query output +1 1 +2 3 + + +-- !query +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 +2 3 3 +7 7 7 + + +-- !query +select getbit(11L, 3), getbit(11L, 2), getbit(11L, 1), getbit(11L, 0) +-- !query schema +struct +-- !query output +1 0 1 1 + + +-- !query +select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(cast(11L / 1 AS long), 1 - 1) +-- !query schema +struct +-- !query output +1 0 1 1 + + +-- !query +select getbit(11L, 63) +-- !query schema +struct +-- !query output +0 + + +-- !query +select getbit(11L, -1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +(-1 vs. 0) The value of 'pos' argument must be greater than or equal to zero. + + +-- !query +select getbit(11L, 64) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +(64 vs. 64) The value of 'pos' argument must not exceed the number of bits in 'x' - 1. diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out new file mode 100644 index 000000000000..1221be7d5b9c --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out @@ -0,0 +1,1243 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table char_tbl(c char(5), v varchar(6)) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl + + +-- !query +desc formatted char_tbl c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +create table char_tbl2 using parquet as select * from char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +show create table char_tbl2 +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl2 ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +desc formatted char_tbl2 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl2 + + +-- !query +desc formatted char_tbl2 c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +create table char_tbl3 like char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl3 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl3 + + +-- !query +desc formatted char_tbl3 c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_tbl3 +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl3 ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +create view char_view as select * from char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl +View Original Text select * from char_tbl +View Catalog and Namespace spark_catalog.default +View Query Output Columns [c, v] + + +-- !query +desc formatted char_view c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_view +-- !query schema +struct +-- !query output +CREATE VIEW default.char_view ( + c, + v) +AS select * from char_tbl + + +-- !query +alter table char_tbl rename to char_tbl1 +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter table char_tbl1 change column c type char(6) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_SUPPORTED_CHANGE_COLUMN", + "messageParameters" : { + "newName" : "`c`", + "newType" : "\"CHAR(6)\"", + "originName" : "`c`", + "originType" : "\"CHAR(5)\"", + "table" : "`spark_catalog`.`default`.`char_tbl1`" + } +} + + +-- !query +alter table char_tbl1 change column c type char(5) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter table char_tbl1 add columns (d char(5)) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view as select * from char_tbl2 +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Catalog and Namespace spark_catalog.default +View Query Output Columns [c, v] + + +-- !query +alter table char_tbl1 SET TBLPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Table Properties [yes=no] +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view SET TBLPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Catalog and Namespace spark_catalog.default +View Query Output Columns [c, v] +Table Properties [yes=no] + + +-- !query +alter table char_tbl1 UNSET TBLPROPERTIES('yes') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view UNSET TBLPROPERTIES('yes') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Catalog and Namespace spark_catalog.default +View Query Output Columns [c, v] + + +-- !query +alter table char_tbl1 SET SERDEPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 +Storage Properties [yes=no] + + +-- !query +create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part add partition (v2='ke', c2='nt') location 'loc1' +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part partition (v2='ke') rename to partition (v2='nt') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } +} + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part partition (v2='ke', c2='nt') set location 'loc2' +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +MSCK REPAIR TABLE char_part +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +create temporary view str_view as select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table char_tbl4(c7 char(7), c8 char(8), v varchar(6), s string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into char_tbl4 select c, c, v, c from str_view +-- !query schema +struct<> +-- !query output + + + +-- !query +select c7, c8, v, s from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = c8 +-- !query schema +struct +-- !query output +N N N N +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = v +-- !query schema +struct +-- !query output + + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = s +-- !query schema +struct +-- !query output +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = 'NetEase ' +-- !query schema +struct +-- !query output +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where v = 'Spark ' +-- !query schema +struct +-- !query output +NetEa NetEa Spark NetEa + + +-- !query +select c7, c8, v, s from char_tbl4 order by c7 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +NULL NULL S NULL +N N N N +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 order by v +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +N N N N +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEas NetEas Spark NetEas +NetEa NetEa Spark NetEa +NetEase NetEase Spark- NetEase + + +-- !query +select ascii(c7), ascii(c8), ascii(v), ascii(s) from char_tbl4 +-- !query schema +struct +-- !query output +78 78 78 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +NULL NULL 83 NULL +NULL NULL NULL NULL + + +-- !query +select base64(c7), base64(c8), base64(v), ascii(s) from char_tbl4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +NULL NULL Uw== NULL +TiAgICAgIA== TiAgICAgICA= TiA= 78 +TmUgICAgIA== TmUgICAgICA= U3A= 78 +TmV0ICAgIA== TmV0ICAgICA= U3BhICA= 78 +TmV0RSAgIA== TmV0RSAgICA= U3Bhcg== 78 +TmV0RWEgIA== TmV0RWEgICA= U3Bhcmsg 78 +TmV0RWFzIA== TmV0RWFzICA= U3Bhcms= 78 +TmV0RWFzZQ== TmV0RWFzZSA= U3Bhcmst 78 + + +-- !query +select bit_length(c7), bit_length(c8), bit_length(v), bit_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +56 64 16 16 +56 64 16 8 +56 64 32 32 +56 64 40 40 +56 64 40 56 +56 64 48 48 +56 64 48 56 +NULL NULL 8 NULL +NULL NULL NULL NULL + + +-- !query +select char_length(c7), char_length(c8), char_length(v), char_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +7 8 2 1 +7 8 2 2 +7 8 4 4 +7 8 5 5 +7 8 5 7 +7 8 6 6 +7 8 6 7 +NULL NULL 1 NULL +NULL NULL NULL NULL + + +-- !query +select octet_length(c7), octet_length(c8), octet_length(v), octet_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +7 8 2 1 +7 8 2 2 +7 8 4 4 +7 8 5 5 +7 8 5 7 +7 8 6 6 +7 8 6 7 +NULL NULL 1 NULL +NULL NULL NULL NULL + + +-- !query +select concat_ws('|', c7, c8), concat_ws('|', c7, v), concat_ws('|', c7, s), concat_ws('|', v, s) from char_tbl4 +-- !query schema +struct +-- !query output + + S S +N |N N |N N |N N |N +Ne |Ne Ne |Sp Ne |Ne Sp|Ne +Net |Net Net |Spa Net |Net Spa |Net +NetE |NetE NetE |Spar NetE |NetE Spar|NetE +NetEa |NetEa NetEa |Spark NetEa |NetEa Spark |NetEa +NetEas |NetEas NetEas |Spark NetEas |NetEas Spark|NetEas +NetEase|NetEase NetEase|Spark- NetEase|NetEase Spark-|NetEase + + +-- !query +select concat(c7, c8), concat(c7, v), concat(c7, s), concat(v, s) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N N N N +NULL NULL NULL NULL +NULL NULL NULL NULL +Ne Ne Ne Sp Ne Ne SpNe +Net Net Net Spa Net Net Spa Net +NetE NetE NetE Spar NetE NetE SparNetE +NetEa NetEa NetEa Spark NetEa NetEa Spark NetEa +NetEas NetEas NetEas Spark NetEas NetEas SparkNetEas +NetEaseNetEase NetEaseSpark- NetEaseNetEase Spark-NetEase + + +-- !query +select like(c7, 'Ne _'), like(c8, 'Ne _') from char_tbl4 +-- !query schema +struct +-- !query output +NULL NULL +NULL NULL +false false +false false +false false +false false +false false +false false +false true + + +-- !query +select like(v, 'Spark_') from char_tbl4 +-- !query schema +struct +-- !query output +NULL +false +false +false +false +false +false +true +true + + +-- !query +select c7 = c8, upper(c7) = upper(c8), lower(c7) = lower(c8) from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = c8):boolean,(upper(c7) = upper(c8)):boolean,(lower(c7) = lower(c8)):boolean> +-- !query output +true false false + + +-- !query +select c7 = s, upper(c7) = upper(s), lower(c7) = lower(s) from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = s):boolean,(upper(c7) = upper(s)):boolean,(lower(c7) = lower(s)):boolean> +-- !query output +true true true + + +-- !query +select c7 = 'NetEase', upper(c7) = upper('NetEase'), lower(c7) = lower('NetEase') from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = NetEase):boolean,(upper(c7) = upper(NetEase)):boolean,(lower(c7) = lower(NetEase)):boolean> +-- !query output +true true true + + +-- !query +select printf('Hey, %s%s%s%s', c7, c8, v, s) from char_tbl4 +-- !query schema +struct +-- !query output +Hey, N N N N +Hey, Ne Ne SpNe +Hey, Net Net Spa Net +Hey, NetE NetE SparNetE +Hey, NetEa NetEa Spark NetEa +Hey, NetEas NetEas SparkNetEas +Hey, NetEaseNetEase Spark-NetEase +Hey, nullnullSnull +Hey, nullnullnullnull + + +-- !query +select repeat(c7, 2), repeat(c8, 2), repeat(v, 2), repeat(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N N NN +NULL NULL NULL NULL +NULL NULL SS NULL +Ne Ne Ne Ne SpSp NeNe +Net Net Net Net Spa Spa Net Net +NetE NetE NetE NetE SparSpar NetENetE +NetEa NetEa NetEa NetEa Spark Spark NetEa NetEa +NetEas NetEas NetEas NetEas SparkSpark NetEas NetEas +NetEaseNetEase NetEase NetEase Spark-Spark- NetEaseNetEase + + +-- !query +select replace(c7, 'Net', 'Apache'), replace(c8, 'Net', 'Apache'), replace(v, 'Spark', 'Kyuubi'), replace(s, 'Net', 'Apache') from char_tbl4 +-- !query schema +struct +-- !query output +Apache Apache Spa Apache +ApacheE ApacheE Spar ApacheE +ApacheEa ApacheEa Kyuubi ApacheEa +ApacheEas ApacheEas Kyuubi ApacheEas +ApacheEase ApacheEase Kyuubi- ApacheEase +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne + + +-- !query +select rpad(c7, 10), rpad(c8, 5), rpad(v, 5), rpad(s, 5) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEa Spark NetEa +NetEase NetEa Spark NetEa + + +-- !query +select rtrim(c7), rtrim(c8), rtrim(v), rtrim(s) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select split(c7, 'e'), split(c8, 'e'), split(v, 'a'), split(s, 'e') from char_tbl4 +-- !query schema +struct,split(c8, e, -1):array,split(v, a, -1):array,split(s, e, -1):array> +-- !query output +NULL NULL NULL NULL +NULL NULL ["S"] NULL +["N "] ["N "] ["N "] ["N"] +["N"," "] ["N"," "] ["Sp"] ["N",""] +["N","t "] ["N","t "] ["Sp"," "] ["N","t "] +["N","tE "] ["N","tE "] ["Sp","r"] ["N","tE"] +["N","tEa "] ["N","tEa "] ["Sp","rk "] ["N","tEa "] +["N","tEas "] ["N","tEas "] ["Sp","rk"] ["N","tEas "] +["N","tEas",""] ["N","tEas"," "] ["Sp","rk-"] ["N","tEas",""] + + +-- !query +select substring(c7, 2), substring(c8, 2), substring(v, 3), substring(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output + +NULL NULL NULL +NULL NULL NULL NULL +e e e +et et a et +etE etE ar etE +etEa etEa ark etEa +etEas etEas ark etEas +etEase etEase ark- etEase + + +-- !query +select left(c7, 2), left(c8, 2), left(v, 3), left(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne + + +-- !query +select right(c7, 2), right(c8, 2), right(v, 3), right(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output + N N + Sp Ne + a + par tE + rk a +NULL NULL NULL NULL +NULL NULL S NULL +s ark s +se e rk- se + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select typeof(c7), typeof(c8), typeof(v), typeof(s) from char_tbl4 limit 1 +-- !query schema +struct +-- !query output +string string string string + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select cast(c7 as char(1)), cast(c8 as char(10)), cast(v as char(1)), cast(v as varchar(1)), cast(s as char(5)) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N +NULL NULL NULL NULL NULL +NULL NULL S S NULL +Ne Ne Sp Sp Ne +Net Net Spa Spa Net +NetE NetE Spar Spar NetE +NetEa NetEa Spark Spark NetEa +NetEas NetEas Spark Spark NetEas +NetEase NetEase Spark- Spark- NetEase + + +-- !query +drop table char_tbl1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl2 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl3 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl4 +-- !query schema +struct<> +-- !query output + + + +-- !query +select ascii('§'), ascii('÷'), ascii('×10') +-- !query schema +struct +-- !query output +167 247 215 + + +-- !query +select chr(167), chr(247), chr(215) +-- !query schema +struct +-- !query output +§ ÷ × + + +-- !query +SELECT to_varchar(78.12, '$99.99') +-- !query schema +struct +-- !query output +$78.12 + + +-- !query +SELECT to_varchar(111.11, '99.9') +-- !query schema +struct +-- !query output +##.# + + +-- !query +SELECT to_varchar(12454.8, '99,999.9S') +-- !query schema +struct +-- !query output +12,454.8+ diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/count.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/count.sql.out new file mode 100644 index 000000000000..714c95cdd3c7 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/count.sql.out @@ -0,0 +1,214 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData +-- !query schema +struct +-- !query output +7 7 0 5 5 4 7 + + +-- !query +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData +-- !query schema +struct +-- !query output +1 0 2 2 2 6 + + +-- !query +SELECT count(a, b), count(b, a), count(testData.*, testData.*) FROM testData +-- !query schema +struct +-- !query output +4 4 4 + + +-- !query +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*, testData.*) +FROM testData +-- !query schema +struct +-- !query output +3 3 3 3 + + +-- !query +SELECT count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 2,3) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData +-- !query schema +struct +-- !query output +1 2 + + +-- !query +set spark.sql.legacy.allowParameterlessCount=true +-- !query schema +struct +-- !query output +spark.sql.legacy.allowParameterlessCount true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding + + +-- !query +SELECT count() FROM testData +-- !query schema +struct +-- !query output +0 + + +-- !query +set spark.sql.legacy.allowParameterlessCount=false +-- !query schema +struct +-- !query output +spark.sql.legacy.allowParameterlessCount false + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT count() FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITH_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "expectedNum" : " >= 1", + "functionName" : "`count`", + "legacyConfKey" : "\"spark.sql.legacy.allowParameterlessCount\"", + "legacyConfValue" : "\"true\"", + "legacyNum" : "0" + } +} + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=true +-- !query schema +struct +-- !query output +spark.sql.legacy.allowStarWithSingleTableIdentifierInCount true + + +-- !query +SELECT count(testData.*) FROM testData +-- !query schema +struct +-- !query output +4 + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=false +-- !query schema +struct +-- !query output +spark.sql.legacy.allowStarWithSingleTableIdentifierInCount false + + +-- !query +SELECT count(testData.*) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1021", + "messageParameters" : { + "targetString" : "testData" + } +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/cte.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/cte.sql.out new file mode 100644 index 000000000000..9b7b6f28089e --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/cte.sql.out @@ -0,0 +1,595 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH s AS (SELECT 1 FROM s) SELECT * FROM s +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 26, + "fragment" : "s" + } ] +} + + +-- !query +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 34, + "fragment" : "r" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 + + +-- !query +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "s2" + } ] +} + + +-- !query +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 +-- !query schema +struct +-- !query output +0 2 +0 2 +1 2 +1 2 + + +-- !query +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 1 +0 1 +1 0 +1 0 +1 0 +1 0 +1 1 +1 1 +1 1 +1 1 + + +-- !query +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2 +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t() AS (SELECT 1) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0038", + "messageParameters" : { + "duplicateNames" : "'t'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "WITH\n t(x) AS (SELECT 1),\n t(x) AS (SELECT 2)\nSELECT * FROM t" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM non_existing_table) +SELECT 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`non_existing_table`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 43, + "fragment" : "non_existing_table" + } ] +} + + +-- !query +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT random() FROM range(1, 5)) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query schema +struct +-- !query output +4 + + +-- !query +WITH w1(c1) AS + (WITH w2(c2) AS + (WITH w3(c3) AS + (WITH w4(c4) AS + (WITH w5(c5) AS + (WITH w6(c6) AS + (WITH w7(c7) AS + (WITH w8(c8) AS + (SELECT 1) + SELECT * FROM w8) + SELECT * FROM w7) + SELECT * FROM w6) + SELECT * FROM w5) + SELECT * FROM w4) + SELECT * FROM w3) + SELECT * FROM w2) +SELECT * FROM w1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( WITH cte(foo) AS ( VALUES(id) ) + SELECT (SELECT foo FROM cte) ) +FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "messageParameters" : { + "expr" : "\"outer(t.id)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 37, + "fragment" : "id" + } ] +} + + +-- !query +WITH same_name AS (SELECT 42) +SELECT * FROM same_name, (SELECT 10) AS same_name +-- !query schema +struct<42:int,10:int> +-- !query output +42 10 + + +-- !query +WITH same_name(x) AS (SELECT 42) +SELECT same_name.x FROM (SELECT 10) AS same_name(x), same_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`same_name`.`x`", + "referenceNames" : "[`same_name`.`x`, `same_name`.`x`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 51, + "fragment" : "same_name.x" + } ] +} + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +WITH q AS (SELECT 'foo' AS x) +SELECT x, typeof(x) FROM q +-- !query schema +struct +-- !query output +foo string + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +with cte as (select id as id_alias from t) +select id from cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`", + "proposal" : "`id_alias`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "id" + } ] +} + + +-- !query +with r1 as (select * from r2), + r2 as (select 1) +select 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "r2" + } ] +} + + +-- !query +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q) +-- !query schema +struct +-- !query output +2 + + +-- !query +select (with q as (select 1 x) select * from q) +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 in (with q as (select 1) select * from q) +-- !query schema +struct<(1 IN (listquery())):boolean> +-- !query output +true + + +-- !query +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q), + q +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`q`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 68, + "stopIndex" : 68, + "fragment" : "q" + } ] +} + + +-- !query +WITH T1 as (select 1 a) +select * +from + T1 x, + (WITH T1 as (select 2 b) select * from T1) y, + T1 z +-- !query schema +struct +-- !query output +1 2 1 + + +-- !query +WITH TTtt as (select 1 a), + `tTTt_2` as (select 2 a) +select * +from + (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) +-- !query schema +struct +-- !query output +3 2 + + +-- !query +select + (WITH q AS (select T.x) select * from q) +from (select 1 x, 2 y) T +-- !query schema +struct +-- !query output +1 + + +-- !query +select + (WITH q AS (select 3 z) select x + t.y + z from q) +from (select 1 x, 2 y) T +-- !query schema +struct +-- !query output +6 + + +-- !query +WITH q1 as (select 1 x) +select * from + (with q2 as (select * from q1) select * from q2) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH q1 as (select 1 x) +select * from + (with q1 as (select x+1 from q1) select * from q1) +-- !query schema +struct<(x + 1):int> +-- !query output +2 + + +-- !query +with cte1 as (select 42), cte1 as (select 42) select * FROM cte1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0038", + "messageParameters" : { + "duplicateNames" : "'cte1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "with cte1 as (select 42), cte1 as (select 42) select * FROM cte1" + } ] +} + + +-- !query +with cte1 as (Select id as j from t) +select * from cte1 where j = (select max(j) from cte1 as cte2) +-- !query schema +struct +-- !query output +2 + + +-- !query +with cte AS (SELECT * FROM va) SELECT * FROM cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`va`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 29, + "fragment" : "va" + } ] +} + + +-- !query +with cte as (select * from cte) select * from cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 30, + "fragment" : "cte" + } ] +} + + +-- !query +DROP VIEW IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out new file mode 100644 index 000000000000..5e2a4d61e118 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out @@ -0,0 +1,2154 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 + + +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !query +select make_date(2019, 1, 1), make_date(12, 12, 12) +-- !query schema +struct +-- !query output +2019-01-01 0012-12-12 + + +-- !query +select make_date(2000, 13, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select make_date(2000, 1, 33) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date'015' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select current_date() = current_date() +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select curdate(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + +-- !query +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 2016-12-31 + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +7 5 7 NULL 6 6 6 + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +5 3 5 NULL 4 4 4 + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1500 1582 1582 1582 + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 10 10 10 + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 288 288 288 + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day("2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_add('2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_add" + } +} + + +-- !query +select date_add(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_sub" + } +} + + +-- !query +select date_sub(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_add('2011-11-11', int_str)" + } ] +} + + +-- !query +select date_sub('2011-11-11', int_str) from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_sub('2011-11-11', int_str)" + } ] +} + + +-- !query +select date_add(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date '2011-11-11' + 1E1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select 7S + date '2001-09-28' +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select date '2001-10-01' - 7 +-- !query schema +struct +-- !query output +2001-09-24 + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "date '2001-10-01' - '2001-09-28'" + } ] +} + + +-- !query +select '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(2001-10-01 - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null - date '2019-10-06' +-- !query schema +struct<(NULL - DATE '2019-10-06'):interval day> +-- !query output +NULL + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query schema +struct<(date_str - DATE '2001-09-28'):interval day> +-- !query output +3696 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "date '2001-09-28' - date_str" + } ] +} + + +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "2", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null + date'2011-11-11' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query schema +struct +-- !query output +2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09 + + +-- !query +select to_date('26/October/2015', 'dd/MMMMM/yyyy') +-- !query schema +struct +-- !query output +2015-10-26 + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"d":2015-10-26} + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"d":2015-10-26} + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.124001 + + +-- !query +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.455 + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 01:03:01 + + +-- !query +select date_add(MINUTE, -100, date'2022-02-25') +-- !query schema +struct +-- !query output +2022-02-24 22:20:00 + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 00:02:03 + + +-- !query +select date_add(DAY, 367, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-27 00:00:00 + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-28 01:02:03 + + +-- !query +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-25 01:02:03 + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-05-25 00:00:00 + + +-- !query +select date_add(YEAR, 1, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-25 00:00:00 + + +-- !query +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`dateadd`", + "invalidValue" : "'MICROSECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')" + } ] +} + + +-- !query +select date_add('QUARTER', 5, date'2022-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_add`", + "invalidValue" : "'QUARTER'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "date_add('QUARTER', 5, date'2022-02-25')" + } ] +} + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query schema +struct +-- !query output +1001 + + +-- !query +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query schema +struct +-- !query output +58 + + +-- !query +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query schema +struct +-- !query output +-100 + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query schema +struct +-- !query output +367 + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query schema +struct +-- !query output +-4 + + +-- !query +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query schema +struct +-- !query output +5 + + +-- !query +select date_diff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query schema +struct +-- !query output +1 + + +-- !query +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_diff`", + "invalidValue" : "'MILLISECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 103, + "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')" + } ] +} + + +-- !query +select datediff('YEAR', date'2022-02-25', date'2023-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`datediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')" + } ] +} + + +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +Could not convert Timestamp(92233720368547, 758000000) to microseconds + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +Could not convert Timestamp(-92233720368548, 242000000) to microseconds + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" + } ] +} + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "str - timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "2", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":2015-10-26 00:00:00} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":2015-10-26 00:00:00} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out new file mode 100644 index 000000000000..caf6605c3bbd --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out @@ -0,0 +1,111 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('2019-366', 'yyyy-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "datetime" : "'2018-366'" + } +} + + +-- !query +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as date) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out new file mode 100644 index 000000000000..32f3216d8c02 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out @@ -0,0 +1,79 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out new file mode 100644 index 000000000000..32f3216d8c02 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out @@ -0,0 +1,79 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 000000000000..b968b4e09fac --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,524 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, sum(b) from data group by 1 +-- !query schema +struct +-- !query output +1 3 +2 3 +3 3 + + +-- !query +select 1, 2, sum(b) from data group by 1, 2 +-- !query schema +struct<1:int,2:int,sum(b):bigint> +-- !query output +1 2 9 + + +-- !query +select a, 1, sum(b) from data group by a, 1 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, 1, sum(b) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, b + 2, count(2) from data group by a, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select sum(b) from data group by 1 + 0 +-- !query schema +struct +-- !query output +9 + + +-- !query +select a, b from data group by -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b from data group by 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] +} + + +-- !query +select a, b from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) + 2 from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query schema +struct +-- !query output +1 0.5488135024422883 1 +1 0.7151893651681639 2 +2 0.5448831775801376 2 +2 0.6027633705776989 1 +3 0.4236547969336536 1 +3 0.6458941151817286 2 + + +-- !query +select * from data group by a, b, 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "STAR_GROUP_BY_POS", + "sqlState" : "0A000" +} + + +-- !query +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select a, a AS k, count(b) from data group by k, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 2 2 +3 3 2 + + +-- !query +select a, b, count(1) from data group by cube(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by cube(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with cube +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with rollup +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by a, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(-1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(1, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.groupByOrdinal false + + +-- !query +select sum(b) from data group by -1 +-- !query schema +struct +-- !query output +9 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out new file mode 100644 index 000000000000..db0b74cd6ac7 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out @@ -0,0 +1,1145 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT a, COUNT(b) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b):bigint> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa) +-- !query schema +struct +-- !query output +2 +2 +2 +3 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" + } ] +} + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "count(testdata.b)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "COUNT(b)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 48, + "fragment" : "k" + } ] +} + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT id FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(max(id) > 0)\"", + "expressionList" : "max(id)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0" + } ] +} + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true false true + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct +-- !query output +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "every(1)" + } ] +} + + +-- !query +SELECT some(1S) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "some(1S)" + } ] +} + + +-- !query +SELECT any(1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "any(1L)" + } ] +} + + +-- !query +SELECT every("true") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT bool_and(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_and(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_and(1.0)" + } ] +} + + +-- !query +SELECT bool_or(1.0D) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_or(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_or(1.0D)" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(NOT (a IS NULL)):boolean,c:bigint> +-- !query output +false 2 +true 7 + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint> +-- !query output +0.5488135024422883 7 +1.0 2 + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query schema +struct>,histogram_3:array>,histogram_5:array>,histogram_10:array>> +-- !query output +[{"x":12,"y":26.0},{"x":38,"y":24.0}] [{"x":9,"y":20.0},{"x":25,"y":11.0},{"x":40,"y":19.0}] [{"x":5,"y":11.0},{"x":14,"y":8.0},{"x":22,"y":7.0},{"x":30,"y":10.0},{"x":43,"y":14.0}] [{"x":3,"y":6.0},{"x":8,"y":6.0},{"x":13,"y":4.0},{"x":17,"y":3.0},{"x":20,"y":4.0},{"x":25,"y":6.0},{"x":31,"y":7.0},{"x":39,"y":5.0},{"x":43,"y":4.0},{"x":48,"y":5.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":2017-03-01 00:00:00,"y":1.0},{"x":2017-04-01 00:00:00,"y":1.0},{"x":2017-05-01 00:00:00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":100-0,"y":1.0},{"x":110-0,"y":1.0},{"x":120-0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":12 20:04:00.000000000,"y":1.0},{"x":12 21:04:00.000000000,"y":1.0},{"x":12 22:04:00.000000000,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query schema +struct,collect_list(col):array> +-- !query output +[1,2,1] [1,2,1] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query schema +struct,collect_list(b):array> +-- !query output +1 [4,4] [4,4] +2 [3,4] [3,4] + + +-- !query +SELECT mode(a), mode(b) FROM testData +-- !query schema +struct +-- !query output +3 1 + + +-- !query +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a +-- !query schema +struct +-- !query output +NULL 1 +1 1 +2 1 +3 1 + + +-- !query +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c +-- !query schema +struct +-- !query output +0 +2 + + +-- !query +SELECT col1, count(*) AS cnt +FROM VALUES + (0.0), + (-0.0), + (double('NaN')), + (double('NaN')), + (double('Infinity')), + (double('Infinity')), + (-double('Infinity')), + (-double('Infinity')) +GROUP BY col1 +ORDER BY col1 +-- !query schema +struct +-- !query output +-Infinity 2 +0.0 2 +Infinity 2 +NaN 2 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/hll.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/hll.sql.out new file mode 100644 index 000000000000..2167537a756c --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/hll.sql.out @@ -0,0 +1,247 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) +FROM VALUES (50), (60), (60), (60), (75), (100) tab(col) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES ('abc'), ('def'), ('abc'), ('ghi'), ('abc') tab(col) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT hll_sketch_estimate( + hll_union( + hll_sketch_agg(col1), + hll_sketch_agg(col2))) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) + FROM (SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT hll_sketch_agg(col) +FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\" or \"STRING\" or \"BINARY\")", + "sqlExpr" : "\"hll_sketch_agg(col, 12)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "hll_sketch_agg(col)" + } ] +} + + +-- !query +SELECT hll_sketch_agg(col, 2) +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_LG_K", + "messageParameters" : { + "function" : "`hll_sketch_agg`", + "max" : "21", + "min" : "4", + "value" : "2" + } +} + + +-- !query +SELECT hll_sketch_agg(col, 40) +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_LG_K", + "messageParameters" : { + "function" : "`hll_sketch_agg`", + "max" : "21", + "min" : "4", + "value" : "40" + } +} + + +-- !query +SELECT hll_union( + hll_sketch_agg(col1, 12), + hll_sketch_agg(col2, 13)) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_UNION_DIFFERENT_LG_K", + "messageParameters" : { + "function" : "`hll_union`", + "left" : "12", + "right" : "13" + } +} + + +-- !query +SELECT hll_union_agg(sketch, false) +FROM (SELECT hll_sketch_agg(col, 12) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_UNION_DIFFERENT_LG_K", + "messageParameters" : { + "function" : "`hll_union_agg`", + "left" : "12", + "right" : "20" + } +} + + +-- !query +SELECT hll_union(1, 2) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"hll_union(1, 2, false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "hll_union(1, 2)" + } ] +} + + +-- !query +SELECT hll_sketch_estimate(CAST ('abc' AS BINARY)) +-- !query schema +struct<> +-- !query output +org.apache.datasketches.SketchesArgumentException +Possible Corruption: Illegal Family ID: 99 + + +-- !query +SELECT hll_union(CAST ('abc' AS BINARY), CAST ('def' AS BINARY)) +-- !query schema +struct<> +-- !query output +org.apache.datasketches.SketchesArgumentException +Possible Corruption: Illegal Family ID: 99 + + +-- !query +SELECT hll_union_agg(buffer, false) +FROM (SELECT CAST('abc' AS BINARY) AS buffer) +-- !query schema +struct<> +-- !query output +org.apache.datasketches.SketchesArgumentException +Possible Corruption: Illegal Family ID: 99 + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/interval.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/interval.sql.out new file mode 100644 index 000000000000..fff40d1d4d01 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/interval.sql.out @@ -0,0 +1,3665 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') +-- !query schema +struct<((TIMESTAMP '2019-10-15 10:11:12.001002' - DATE '2019-10-15') * 3):interval day to second> +-- !query output +1 06:33:36.003006000 + + +-- !query +select interval 4 month 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} + + +-- !query +select interval 2 years 4 months +-- !query schema +struct +-- !query output +2-4 + + +-- !query +select interval 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<(INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5):interval day to second> +-- !query output +21 00:00:00.000005000 + + +-- !query +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 +-- !query schema +struct<((TIMESTAMP '2019-10-15 00:00:00' - TIMESTAMP '2019-10-14 00:00:00') / 1.5):interval day to second> +-- !query output +0 16:00:00.000000000 + + +-- !query +select interval 2147483647 month * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +integer overflow + + +-- !query +select interval 2147483647 month / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select interval 2147483647 day * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2147483647 day / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2 second * '2' +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select interval 2 second / '2' +-- !query schema +struct<(INTERVAL '02' SECOND / 2):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select interval 2 year * '2' +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select interval 2 year / '2' +-- !query schema +struct<(INTERVAL '2' YEAR / 2):interval year to month> +-- !query output +1-0 + + +-- !query +select interval 2 second * 'a' +-- !query schema +struct<(INTERVAL '02' SECOND * a):interval day to second> +-- !query output +NULL + + +-- !query +select interval 2 second / 'a' +-- !query schema +struct<(INTERVAL '02' SECOND / a):interval day to second> +-- !query output +NULL + + +-- !query +select interval 2 year * 'a' +-- !query schema +struct<(INTERVAL '2' YEAR * a):interval year to month> +-- !query output +NULL + + +-- !query +select interval 2 year / 'a' +-- !query schema +struct<(INTERVAL '2' YEAR / a):interval year to month> +-- !query output +NULL + + +-- !query +select '2' * interval 2 second +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select '2' * interval 2 year +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select 'a' * interval 2 second +-- !query schema +struct<(INTERVAL '02' SECOND * a):interval day to second> +-- !query output +NULL + + +-- !query +select 'a' * interval 2 year +-- !query schema +struct<(INTERVAL '2' YEAR * a):interval year to month> +-- !query output +NULL + + +-- !query +select '2' / interval 2 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'2' / interval 2 second" + } ] +} + + +-- !query +select '2' / interval 2 year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'2' / interval 2 year" + } ] +} + + +-- !query +select interval '2 seconds' / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "interval '2 seconds' / 0" + } ] +} + + +-- !query +select interval '2 seconds' / null +-- !query schema +struct<(INTERVAL '02' SECOND / NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2 seconds' * null +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null * interval '2 seconds' +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2' year / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval '2' year / 0" + } ] +} + + +-- !query +select interval '2' year / null +-- !query schema +struct<(INTERVAL '2' YEAR / NULL):interval year to month> +-- !query output +NULL + + +-- !query +select interval '2' year * null +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select null * interval '2' year +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select 2 / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' year" + } ] +} + + +-- !query +select 2 / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' hour" + } ] +} + + +-- !query +select null / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' year" + } ] +} + + +-- !query +select null / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' hour" + } ] +} + + +-- !query +select -interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select -interval '-1 year 1 month' +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select -interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select -interval -1 year 1 month +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select +interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select +interval '-1 year 1 month' +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select +interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select +interval -1 year 1 month +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select interval -'1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval -'-1-1' year to month +-- !query schema +struct +-- !query output +1-1 + + +-- !query +select interval +'-1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval - '1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +-1 02:03:04.001000000 + + +-- !query +select interval +'1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'-1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'1' year +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +select interval -'-1' year +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select interval -'11' month +-- !query schema +struct +-- !query output +-0-11 + + +-- !query +select interval -'-11' month +-- !query schema +struct +-- !query output +0-11 + + +-- !query +select interval -'1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +select interval -'-1' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval -'23' hour +-- !query schema +struct +-- !query output +-0 23:00:00.000000000 + + +-- !query +select interval -'-23' hour +-- !query schema +struct +-- !query output +0 23:00:00.000000000 + + +-- !query +select interval -'59' minute +-- !query schema +struct +-- !query output +-0 00:59:00.000000000 + + +-- !query +select interval -'-59' minute +-- !query schema +struct +-- !query output +0 00:59:00.000000000 + + +-- !query +select interval -'59' second +-- !query schema +struct +-- !query output +-0 00:00:59.000000000 + + +-- !query +select interval -'-59' second +-- !query schema +struct +-- !query output +0 00:00:59.000000000 + + +-- !query +select make_interval(1) +-- !query schema +struct +-- !query output +1 years + + +-- !query +select make_interval(1, 2) +-- !query schema +struct +-- !query output +1 years 2 months + + +-- !query +select make_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 years 2 months 21 days + + +-- !query +select make_interval(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 years 2 months 25 days + + +-- !query +select make_interval(1, 2, 3, 4, 5) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6, 7.008009) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 34293552 hours 30 minutes 12.123456 seconds + + +-- !query +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select make_dt_interval(1) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2) +-- !query schema +struct +-- !query output +1 02:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 02:03:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3, 4.005006) +-- !query schema +struct +-- !query output +1 02:03:04.005006000 + + +-- !query +select make_dt_interval(1, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1428899 00:30:12.123456000 + + +-- !query +select make_dt_interval(2147483647) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select make_ym_interval(1) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select make_ym_interval(1, 2) +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select make_ym_interval(0, 1) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +select make_ym_interval(178956970, 7) +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +select make_ym_interval(178956970, 8) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +(2147483648 vs. -2147483648) Integer overflow in make_ym_interval(178956970, 8) + + +-- !query +select make_ym_interval(-178956970, -8) +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select make_ym_interval(-178956970, -9) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +(-2147483649 vs. 2147483647) Integer overflow in make_ym_interval(-178956970, -9) + + +-- !query +select cast('1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('+1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('-1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('+ 1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('- 1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('- -1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('- +1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query schema +struct +-- !query output +0 00:00:13.123456000 -0 00:00:13.123456000 + + +-- !query +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} + + +-- !query +select interval 1 year 2 month +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select interval 4 day 5 hour 6 minute 7 seconds +-- !query schema +struct +-- !query output +4 05:06:07.000000000 + + +-- !query +select interval 3 week 8 millisecond 9 microsecond +-- !query schema +struct +-- !query output +21 00:00:00.008009000 + + +-- !query +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} + + +-- !query +select interval '30' year '25' month +-- !query schema +struct +-- !query output +32-1 + + +-- !query +select interval '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct +-- !query output +-98 06:35:00.110013000 + + +-- !query +select interval '0-0' year to month +-- !query schema +struct +-- !query output +0-0 + + +-- !query +select interval '0 0:0:0' day to second +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +select interval '0 0:0:0.1' day to second +-- !query schema +struct +-- !query output +0 00:00:00.100000000 + + +-- !query +select interval '10-9' year to month +-- !query schema +struct +-- !query output +10-9 + + +-- !query +select interval '20 15' day to hour +-- !query schema +struct +-- !query output +20 15:00:00.000000000 + + +-- !query +select interval '20 15:40' day to minute +-- !query schema +struct +-- !query output +20 15:40:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to second +-- !query schema +struct +-- !query output +20 15:40:32.998999000 + + +-- !query +select interval '15:40' hour to minute +-- !query schema +struct +-- !query output +0 15:40:00.000000000 + + +-- !query +select interval '15:40:32.99899999' hour to second +-- !query schema +struct +-- !query output +0 15:40:32.998999000 + + +-- !query +select interval '40:32.99899999' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.998999000 + + +-- !query +select interval '40:32' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.000000000 + + +-- !query +select interval 30 day day +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval 30 days days +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} + + +-- !query +select interval '20 15:40:32.99899999' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} + + +-- !query +select interval '15:40:32.99899999' hour to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} + + +-- !query +select interval '15:40.99899999' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} + + +-- !query +select interval '15:40' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} + + +-- !query +select interval '20 40:32.99899999' minute to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} + + +-- !query +select interval 10 nanoseconds +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0062", + "messageParameters" : { + "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "10 nanoseconds" + } ] +} + + +-- !query +select map(1, interval 1 day, 2, interval 3 week) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:21 00:00:00.000000000} + + +-- !query +select map(1, interval 1 day, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 1 year, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:1-0,2:0-2} + + +-- !query +select map(1, interval 1 month, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:0-1,2:0-2} + + +-- !query +select map(1, interval 1 week, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:7 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 2 millisecond, 3, interval 3 microsecond) +-- !query schema +struct> +-- !query output +{1:0 00:00:00.002000000,3:0 00:00:00.000003000} + + +-- !query +select interval 'interval 3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +select interval '3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +select interval 1 fake_unit +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'fake_unit'", + "hint" : "" + } +} + + +-- !query +select interval 1 year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} + + +-- !query +select interval '1' year to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0028", + "messageParameters" : { + "from" : "year", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} + + +-- !query +select interval '10-9' year to month '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} + + +-- !query +select interval '10-9' year to month '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} + + +-- !query +select interval 1 year '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} + + +-- !query +select interval 1 year '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '10-9' year to month '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} + + +-- !query +select interval '12:11:10' hour to second '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} + + +-- !query +select interval (-30) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 day day day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'day'", + "hint" : ": extra input 'day'" + } +} + + +-- !query +select interval (-30) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 days days days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'days'", + "hint" : ": extra input 'days'" + } +} + + +-- !query +SELECT INTERVAL '178956970-7' YEAR TO MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '178956970-8' YEAR TO MONTH +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Error parsing interval year-month string: integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} + + +-- !query +SELECT INTERVAL '-178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +SELECT INTERVAL -'178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day +-- !query schema +struct<(INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' MONTH):interval year to month,(INTERVAL '2' YEAR - INTERVAL '3-3' YEAR TO MONTH):interval year to month,(INTERVAL '99 11:22:33.123456' DAY TO SECOND + INTERVAL '10 09:08' DAY TO MINUTE):interval day to second,(INTERVAL '22:33.123456' MINUTE TO SECOND - INTERVAL '10' DAY):interval day to second> +-- !query output +2-5 -1-3 109 20:30:33.123456000 -9 23:37:26.876544000 + + +-- !query +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" + } ] +} + + +-- !query +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour +-- !query schema +struct<(INTERVAL '2' YEAR + NULL):interval year,(INTERVAL '2' YEAR - NULL):interval year,(INTERVAL '02' HOUR + NULL):interval hour,(INTERVAL '02' HOUR - NULL):interval hour,(NULL + INTERVAL '2' YEAR):interval year,(NULL - INTERVAL '2' YEAR):interval year,(NULL + INTERVAL '02' HOUR):interval hour,(NULL - INTERVAL '02' HOUR):interval hour> +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL + + +-- !query +select interval '2' year + '3-3' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "interval '2' year + '3-3'" + } ] +} + + +-- !query +select interval '2' year - '4' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - '4'" + } ] +} + + +-- !query +select '4 11:11' - interval '4 22:12' day to minute +-- !query schema +struct<4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE:string> +-- !query output +NULL + + +-- !query +select '4 12:12:12' + interval '4 22:12' day to minute +-- !query schema +struct<4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE:string> +-- !query output +NULL + + +-- !query +create temporary view interval_view as select '1' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select interval '2' year + str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" + } ] +} + + +-- !query +select interval '2' year - str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" + } ] +} + + +-- !query +select str - interval '4 22:12' day to minute from interval_view +-- !query schema +struct +-- !query output +NULL + + +-- !query +select str + interval '4 22:12' day to minute from interval_view +-- !query schema +struct +-- !query output +NULL + + +-- !query +select interval '2-2' year to month + interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month + interval '3' day" + } ] +} + + +-- !query +select interval '3' day + interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day + interval '2-2' year to month" + } ] +} + + +-- !query +select interval '2-2' year to month - interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month - interval '3' day" + } ] +} + + +-- !query +select interval '3' day - interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day - interval '2-2' year to month" + } ] +} + + +-- !query +select 1 - interval '2' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "1 - interval '2' second" + } ] +} + + +-- !query +select 1 + interval '2' month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "1 + interval '2' month" + } ] +} + + +-- !query +select interval '2' second + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' second + 1" + } ] +} + + +-- !query +select interval '2' month - 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "interval '2' month - 1" + } ] +} + + +-- !query +select interval '\t interval 1 day' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval \t 1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval\t1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1\t' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1 ' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '2-2\t' year to month +-- !query schema +struct +-- !query output +2-2 + + +-- !query +select interval '-\t2-2\t' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match year-month format of `[+|-]y-m`, `INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH` when cast to interval year to month: -\t2-2\t" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} + + +-- !query +select interval '\n0 12:34:46.789\t' day to second +-- !query schema +struct +-- !query output +0 12:34:46.789000000 + + +-- !query +select interval '\n-\t10\t 12:34:46.789\t' day to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} + + +-- !query +select interval '中文 interval 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'中文 interval 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} + + +-- !query +select interval 'interval中文 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval中文 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} + + +-- !query +select interval 'interval 1中文day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval 1中文day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} + + +-- !query +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", + "sqlState" : "22015", + "messageParameters" : { + "alternative" : "", + "message" : "integer overflow" + } +} + + +-- !query +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", + "sqlState" : "22015", + "messageParameters" : { + "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" + } +} + + +-- !query +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW", + "sqlState" : "22015", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "message" : "integer overflow" + } +} + + +-- !query +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute') +-- !query schema +struct,from_csv(1, 1):struct,to_csv(from_csv(1, 1 day)):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE))):struct> +-- !query output +{"a":1,"b":1 days} {"a":1,"b":1 00:00:00.000000000} 1,1 days 1,INTERVAL '1' DAY INTERVAL '32' HOUR,INTERVAL '70' MINUTE {"a":1 08:00:00.000000000,"b":0 01:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month') +-- !query schema +struct,from_csv(1, 1):struct,to_json(from_json({"a":"1 days"})):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH))):struct> +-- !query output +{"a":1 days} {"a":1,"b":1-0} {"a":"1 days"} 1,INTERVAL '1' YEAR INTERVAL '32' YEAR,INTERVAL '10' MONTH {"a":32-0,"b":0-10} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE)):string,from_json(to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE))):struct> +-- !query output +{"a":1 00:00:00.000000000} {"a":"INTERVAL '1' DAY"} {"a":"INTERVAL '100 02:10' DAY TO MINUTE"} {"a":100 02:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '32-10' YEAR TO MONTH)):string,from_json(to_json(map(a, INTERVAL '32-10' YEAR TO MONTH))):struct> +-- !query output +{"a":1-0} {"a":"INTERVAL '1' YEAR"} {"a":"INTERVAL '32-10' YEAR TO MONTH"} {"a":32-10} + + +-- !query +select interval '+' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} + + +-- !query +select interval '+.' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+.'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} + + +-- !query +select interval '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} + + +-- !query +select interval '1.2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1.2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} + + +-- !query +select interval '- 2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'- 2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} + + +-- !query +select interval '1 day -' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day -'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} + + +-- !query +select interval '1 day 1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day 1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} + + +-- !query +select interval '1 day 2' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} + + +-- !query +select interval 'interval 1' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} + + +-- !query +select interval '-\t 1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 2):interval year to month> +-- !query output +-89478485-4 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 5):interval year to month> +-- !query output +-35791394-2 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in integral divide" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in integral divide" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +not in range + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2):interval day to second> +-- !query output +-53375995 14:00:27.387904000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5):interval day to second> +-- !query output +-21350398 05:36:10.955162000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in integral divide" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in integral divide" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +not in range + + +-- !query +SELECT INTERVAL '106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '-106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '-106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '178956970' YEAR +-- !query schema +struct +-- !query output +178956970-0 + + +-- !query +SELECT INTERVAL '-178956970' YEAR +-- !query schema +struct +-- !query output +-178956970-0 + + +-- !query +SELECT INTERVAL '2147483647' MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '-2147483647' MONTH +-- !query schema +struct +-- !query output +-178956970-7 + + +-- !query +SELECT INTERVAL '106751991' DAY +-- !query schema +struct +-- !query output +106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991' DAY +-- !query schema +struct +-- !query output +-106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788' HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788' HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '153722867280' MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-153722867280' MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '54.775807' SECOND +-- !query schema +struct +-- !query output +0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '-54.775807' SECOND +-- !query schema +struct +-- !query output +-0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR +-- !query schema +struct<(INTERVAL '1' DAY > INTERVAL '01' HOUR):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND +-- !query schema +struct<(INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH +-- !query schema +struct<(INTERVAL '1' YEAR < INTERVAL '1' MONTH):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH +-- !query schema +struct<(INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' < INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' = INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' > INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' < INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' = INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' > INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct> +-- !query output +[1-0,0-1] + + +-- !query +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct> +-- !query output +[1 00:00:00.000000000,0 01:01:00.000000000] + + +-- !query +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`array`", + "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`coalesce`", + "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT abs(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +10-0 + + +-- !query +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND) +-- !query schema +struct +-- !query output +1 02:03:04.123000000 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '1' YEAR):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '-1' MONTH):bigint> +-- !query output +-13 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '1' DAY):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '-01' HOUR):bigint> +-- !query output +-30 + + +-- !query +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] +} + + +-- !query +SELECT signum(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' MONTH) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0-0' YEAR TO MONTH) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT signum(INTERVAL '-10' DAY) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' HOUR) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10) +-- !query schema +struct +-- !query output +5 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out new file mode 100644 index 000000000000..10c3e5bcc06f --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out @@ -0,0 +1,243 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 1 0 +2 4 3 + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 0 0 +2 3 3 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL 10.0 NULL NULL +2 22.666666666666668 21.25 22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL NULL NULL +2 22.666666666666668 20.0 22.666666666666668 20.0 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 288.66666666666663 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 240.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 200.0 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 0.8314087759815244 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out new file mode 100644 index 000000000000..f64e2a123a33 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,150 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select typeof(null) +-- !query schema +struct +-- !query output +void + + +-- !query +select typeof(true) +-- !query schema +struct +-- !query output +boolean + + +-- !query +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query schema +struct +-- !query output +tinyint smallint int bigint + + +-- !query +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query schema +struct +-- !query output +float double decimal(2,1) + + +-- !query +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query schema +struct +-- !query output +date timestamp interval day + + +-- !query +select typeof(x'ABCD'), typeof('SPARK') +-- !query schema +struct +-- !query output +binary string + + +-- !query +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query schema +struct +-- !query output +array map struct + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'false' is not true! + + +-- !query +SELECT assert_true(boolean(0)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(0 as boolean)' is not true! + + +-- !query +SELECT assert_true(null) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'null' is not true! + + +-- !query +SELECT assert_true(boolean(null)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(null as boolean)' is not true! + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +custom error message + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT raise_error('error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +error message + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +too big: 8 diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out new file mode 100644 index 000000000000..d27551f99a4b --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out @@ -0,0 +1,907 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query schema +struct 0)):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +10.0 15.0 30.0 27.5 + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct 0)):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +0 10.0 NULL 30.0 NULL +1 12.5 12.5 17.5 17.5 +2 17.5 17.5 26.25 26.25 +3 60.0 60.0 60.0 60.0 +4 NULL NULL NULL NULL + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query schema +struct 0)):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +10.0 10.0 30.0 30.0 + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct 0)):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +0 10.0 NULL 30.0 NULL +1 10.0 10.0 20.0 20.0 +2 10.0 10.0 30.0 30.0 +3 60.0 60.0 60.0 60.0 +4 NULL NULL NULL NULL + + +-- !query +SELECT + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct +-- !query output +20.0 20.0 20.0 + + +-- !query +SELECT + k, + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 20.0 20.0 20.0 +1 15.0 15.0 15.0 +2 22.5 22.5 22.5 +3 60.0 60.0 60.0 +4 NULL NULL NULL + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5917.75 5186.0 7381.25 8113.0 +Anthony Bow Accounting 6627 8543.75 8435.0 9746.5 9998.0 +Foon Yue Tseng Sales 6660 8550.75 6660.0 9721.5 10563.0 +Gerard Hernandez SCM 6949 10449.0 10449.0 11303.0 11303.0 +Leslie Jennings IT 8113 5917.75 5186.0 7381.25 8113.0 +Diane Murphy Accounting 8435 8543.75 8435.0 9746.5 9998.0 +William Patterson Accounting 8870 8543.75 8435.0 9746.5 9998.0 +Jeff Firrelli Accounting 8992 8543.75 8435.0 9746.5 9998.0 +Julie Firrelli Sales 9181 8550.75 6660.0 9721.5 10563.0 +Steve Patterson Sales 9441 8550.75 6660.0 9721.5 10563.0 +Mary Patterson Accounting 9998 8543.75 8435.0 9746.5 9998.0 +Loui Bondur SCM 10449 10449.0 10449.0 11303.0 11303.0 +George Vanauf Sales 10563 8550.75 6660.0 9721.5 10563.0 +Barry Jones SCM 10586 10449.0 10449.0 11303.0 11303.0 +Pamela Castillo SCM 11303 10449.0 10449.0 11303.0 11303.0 +Gerard Bondur Accounting 11472 8543.75 8435.0 9746.5 9998.0 +Larry Bott SCM 11798 10449.0 10449.0 11303.0 11303.0 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 119, + "fragment" : "median(salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 152, + "fragment" : "median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5917.75 5186.0 7381.25 8113.0 +Anthony Bow Accounting 6627 8543.75 8435.0 9746.5 9998.0 +Foon Yue Tseng Sales 6660 8550.75 6660.0 9721.5 10563.0 +Gerard Hernandez SCM 6949 10449.0 10449.0 11303.0 11303.0 +Leslie Jennings IT 8113 5917.75 5186.0 7381.25 8113.0 +Diane Murphy Accounting 8435 8543.75 8435.0 9746.5 9998.0 +William Patterson Accounting 8870 8543.75 8435.0 9746.5 9998.0 +Jeff Firrelli Accounting 8992 8543.75 8435.0 9746.5 9998.0 +Julie Firrelli Sales 9181 8550.75 6660.0 9721.5 10563.0 +Steve Patterson Sales 9441 8550.75 6660.0 9721.5 10563.0 +Mary Patterson Accounting 9998 8543.75 8435.0 9746.5 9998.0 +Loui Bondur SCM 10449 10449.0 10449.0 11303.0 11303.0 +George Vanauf Sales 10563 8550.75 6660.0 9721.5 10563.0 +Barry Jones SCM 10586 10449.0 10449.0 11303.0 11303.0 +Pamela Castillo SCM 11303 10449.0 10449.0 11303.0 11303.0 +Gerard Bondur Accounting 11472 8543.75 8435.0 9746.5 9998.0 +Larry Bott SCM 11798 10449.0 10449.0 11303.0 11303.0 + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query schema +struct +-- !query output +Jeff Firrelli Accounting 8992 9998.0 9998.0 9998.0 9998.0 9998.0 +Julie Firrelli Sales 9181 9441.0 9441.0 9441.0 9441.0 9441.0 +Steve Patterson Sales 9441 9441.0 9441.0 9441.0 9441.0 9441.0 +Mary Patterson Accounting 9998 9998.0 9998.0 9998.0 9998.0 9998.0 +Loui Bondur SCM 10449 10944.5 10944.5 10586.0 10944.5 11303.0 +George Vanauf Sales 10563 9441.0 9441.0 9441.0 9441.0 9441.0 +Barry Jones SCM 10586 10944.5 10944.5 10586.0 10944.5 11303.0 +Pamela Castillo SCM 11303 10944.5 10944.5 10586.0 10944.5 11303.0 +Gerard Bondur Accounting 11472 9998.0 9998.0 9998.0 9998.0 9998.0 +Larry Bott SCM 11798 10944.5 10944.5 10586.0 10944.5 11303.0 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query schema +struct +-- !query output +0-10 2-6 + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym), + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:10.000000000 0 00:00:30.000000000 +1 0 00:00:12.500000000 0 00:00:17.500000000 +2 0 00:00:17.500000000 0 00:00:26.250000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:10:00.000000000 0 00:30:00.000000000 +1 0 00:12:30.000000000 0 00:17:30.000000000 +2 0 00:17:30.000000000 0 00:26:15.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query schema +struct +-- !query output +0-10 2-6 + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym), + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:10.000000000 0 00:00:30.000000000 +1 0 00:00:10.000000000 0 00:00:20.000000000 +2 0 00:00:10.000000000 0 00:00:30.000000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:10:00.000000000 0 00:30:00.000000000 +1 0 00:10:00.000000000 0 00:20:00.000000000 +2 0 00:10:00.000000000 0 00:30:00.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL + + +-- !query +SELECT + median(dt), + percentile(dt, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt) +FROM intervals +-- !query schema +struct +-- !query output +1-8 1-8 1-8 + + +-- !query +SELECT + k, + median(ym), + percentile(ym, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY ym) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:20.000000000 0 00:00:20.000000000 0 00:00:20.000000000 +1 0 00:00:15.000000000 0 00:00:15.000000000 0 00:00:15.000000000 +2 0 00:00:22.500000000 0 00:00:22.500000000 0 00:00:22.500000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL NULL + + +-- !query +SELECT + k, + median(dt2), + percentile(dt2, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:20:00.000000000 0 00:20:00.000000000 0 00:20:00.000000000 +1 0 00:15:00.000000000 0 00:15:00.000000000 0 00:15:00.000000000 +2 0 00:22:30.000000000 0 00:22:30.000000000 0 00:22:30.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL NULL + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 0.0 0.0 1.0 1.0 1.0 1.0 1.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 1.0 1.0 1.0 2.0 2.0 2.0 2.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 1.0 1.0 2.0 2.0 3.0 3.0 4.0 4.0 + + +-- !query +SET spark.sql.legacy.percentileDiscCalculation = true +-- !query schema +struct +-- !query output +spark.sql.legacy.percentileDiscCalculation true + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 1.0 1.0 2.0 2.0 2.0 3.0 3.0 4.0 + + +-- !query +SELECT + percentile_cont(b) WITHIN GROUP (ORDER BY a DESC) as p0 +FROM values (12, 0.25), (13, 0.25), (22, 0.25) as v(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"b\"", + "inputName" : "percentage", + "inputType" : "\"DOUBLE\"", + "sqlExpr" : "\"percentile_cont(a, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "percentile_cont(b) WITHIN GROUP (ORDER BY a DESC)" + } ] +} + + +-- !query +SET spark.sql.legacy.percentileDiscCalculation = false +-- !query schema +struct +-- !query output +spark.sql.legacy.percentileDiscCalculation false diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out new file mode 100755 index 000000000000..481bf38623e2 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out @@ -0,0 +1,754 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten +-- !query schema +struct +-- !query output +0 0 0 0.0 +0 2 0 2.0 +0 4 0 4.0 +0 6 0 6.0 +0 8 0 8.0 +1 1 2500 1.0 +1 3 2500 3.0 +1 5 2500 5.0 +1 7 2500 7.0 +1 9 2500 9.0 +2 0 5000 0.0 +2 2 5000 2.0 +2 4 5000 4.0 +2 6 5000 6.0 +2 8 5000 8.0 +3 1 7500 1.0 +3 3 7500 3.0 +3 5 7500 5.0 +3 7 7500 7.0 +3 9 7500 9.0 + + +-- !query +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () +-- !query schema +struct +-- !query output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 0 0 +0 4 0 +1 7 1 +1 9 1 +2 0 2 +3 1 3 +3 3 3 +4 1 1 +5 1 1 + + +-- !query +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +3 4 0 +3 7 1 +4 9 1 + + +-- !query +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +2 4 0 +2 7 1 +3 9 1 + + +-- !query +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.0 0 0 +0.0 0 0 +0.0 0 2 +0.0 1 1 +0.0 1 1 +0.0 1 3 +0.6666666666666666 7 1 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.5 1 1 +0.5 1 1 +0.5 1 3 +0.6666666666666666 0 0 +0.6666666666666666 0 0 +0.75 7 1 +1.0 0 2 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +2 1 1 +2 1 3 +2 3 3 +3 4 0 +3 7 1 +3 9 1 + + +-- !query +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 4 0 +1 1 1 +1 3 3 +1 7 1 +7 9 1 +NULL 0 0 +NULL 0 2 +NULL 1 1 +NULL 1 3 + + +-- !query +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +1 1 1 +3 1 3 +4 0 0 +7 1 1 +9 7 1 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +-1 0 2 +-1 3 3 +-1 4 0 +-1 9 1 +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 + + +-- !query +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 0 0 +0 0 2 +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 3 3 +1 7 1 +1 9 1 + + +-- !query +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 7 1 +1 9 1 +2 0 0 +2 0 0 +2 0 2 +3 3 3 + + +-- !query +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten +-- !query schema +struct +-- !query output +4 0 0 +4 0 0 +4 4 0 +9 1 1 +9 1 1 +9 7 1 +9 9 1 +0 0 2 +3 1 3 +3 3 3 + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two +-- !query schema +struct +-- !query output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 +-- !query schema +struct +-- !query output +2 3 +2 3 +4 1 +4 1 +4 1 +4 1 + + +-- !query +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +136 +22 +22 +24 +24 +51 +82 +87 +92 +92 + + +-- !query +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum +-- !query schema +struct +-- !query output + + + +-- !query +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.0 +0.0 +0.0 +1.0 +1.0 +1.0 +1.0 +2.0 +3.0 +3.0 + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) +-- !query schema +struct +-- !query output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s +-- !query schema +struct +-- !query output + + + +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 +-- !query schema +struct +-- !query output +0 + + +-- !query +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten +-- !query schema +struct +-- !query output +0 9976146 4 +1 10114187 9 +2 10059554 8 +3 9878541 1 +4 9881005 2 +5 9981670 5 +6 9947099 3 +7 10120309 10 +8 9991305 6 +9 10040184 7 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 20 8 +0 2 20 8 +0 4 20 8 +0 6 20 8 +0 8 20 8 +1 1 25 9 +1 3 25 9 +1 5 25 9 +1 7 25 9 +1 9 25 9 +2 0 20 8 +2 2 20 8 +2 4 20 8 +2 6 20 8 +2 8 20 8 +3 1 25 9 +3 3 25 9 +3 5 25 9 +3 7 25 9 +3 9 25 9 + + +-- !query +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 3 3 +10 7 3 +18 2 2 +18 6 2 +33 1 1 +33 5 1 +33 9 1 +45 0 0 +45 4 0 +45 8 0 + + +-- !query +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +0 0 0 +10 3 3 +15 5 1 +23 8 0 +32 9 1 +38 6 2 +39 1 1 +41 2 2 +45 4 0 +7 7 3 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 0 0 +13 2 2 +15 7 3 +22 1 1 +23 3 3 +26 6 2 +29 9 1 +31 8 0 +32 5 1 +7 4 0 + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 0 0 +13 3 3 +15 8 0 +17 5 1 +3 6 2 +4 2 2 +6 1 1 +7 9 1 +8 7 3 +NULL 4 0 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +0 7 3 +10 5 1 +15 8 0 +16 2 2 +16 9 1 +22 6 2 +23 1 1 +7 3 3 +9 4 0 +NULL 0 0 + + +-- !query +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +13 1 1 +22 6 2 +30 9 1 +35 8 0 +38 5 1 +45 0 0 +45 3 3 +45 7 3 +6 4 0 +7 2 2 + + +-- !query +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT * FROM v_window +-- !query schema +struct +-- !query output +1 3 +10 19 +2 6 +3 9 +4 12 +5 15 +6 18 +7 21 +8 24 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +DROP VIEW v_window +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW tenk2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW int4_tbl +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out new file mode 100644 index 000000000000..75fc70c3f74c --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out @@ -0,0 +1,556 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + "messageParameters" : { + "sqlExpr" : "\"CAST(11:00 BST AS TIMESTAMP)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 1698, + "fragment" : "values\n(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'),\n(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'),\n(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'),\n(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'),\n(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'),\n(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'),\n(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'),\n(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54')" + } ] +} + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 4 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +3 9 +31 93 +33 99 +35 68 +5 15 +7 21 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 1 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +3 3 +31 31 +33 33 +35 35 +5 5 +7 7 +9 9 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 2 +1 3 +1 7 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +31 93 +33 99 +35 105 +37 111 +39 117 +41 123 +43 129 +45 135 +47 141 +49 96 +5 13 +7 21 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 3 +1 3 +1 3 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +31 31 +33 33 +35 35 +37 37 +39 39 +41 41 +43 43 +45 45 +47 47 +49 49 +5 5 +7 7 +9 9 + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0 +-- !query schema +struct +-- !query output + + + +-- !query +create table t1 (f1 int, f2 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values (1,1),(1,2),(2,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 108, + "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" + } ] +} + + +-- !query +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 1 +2 2 + + +-- !query +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT rank() OVER (ORDER BY length('abc')) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "messageParameters" : { + "invalidExprSqls" : "\"row_number() OVER (ORDER BY salary ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 84, + "fragment" : "INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10" + } ] +} + + +-- !query +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "messageParameters" : { + "invalidExprSqls" : "\"RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 67, + "fragment" : "GROUP BY 1" + } ] +} + + +-- !query +SELECT * FROM rank() OVER (ORDER BY random()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : ": extra input 'BY'" + } +} + + +-- !query +SELECT range(1, 100) OVER () FROM empsalary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`range`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "range(1, 100) OVER ()" + } ] +} + + +-- !query +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "buckets", + "sqlExpr" : "\"ntile(0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "ntile(0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0L", + "exprName" : "offset", + "sqlExpr" : "\"nth_value(four, 0)\"", + "valueRange" : "(0, 9223372036854775807]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "nth_value(four, 0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +DROP TABLE empsalary +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE datetimes +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out new file mode 100644 index 000000000000..e0c261121e56 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out @@ -0,0 +1,529 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 2.00000 +2 2.50000 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3.3 +2 2.2 +3 NULL +4 NULL + + +-- !query +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n) +-- !query schema +struct +-- !query output +3.00 +5.00 +6.01 + + +-- !query +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 2 +2 1 +3 0 +4 0 + + +-- !query +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 4 +2 3 +3 2 +4 1 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 6 +3 9 +4 7 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + "messageParameters" : { + "sqlExpr" : "\"CAST(nan AS INT)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 92, + "stopIndex" : 145, + "fragment" : "VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/random.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/random.sql.out new file mode 100644 index 000000000000..17e6f871b9c5 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/random.sql.out @@ -0,0 +1,115 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT rand(0) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(3 / 7 AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(NULL) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(NULL AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand(1.0)" + } ] +} + + +-- !query +SELECT randn(0L) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(3 / 7 AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(NULL) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(NULL AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT rand('1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand('1')" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out new file mode 100644 index 000000000000..907ac588feaf --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out @@ -0,0 +1,600 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 1 in regex '\d+' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 1 in regex '\d+' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 2 in regex '\d+' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group -1 in regex '\d+' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct +-- !query output +1a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct +-- !query output +a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 3 in regex '(\d+)([a-z]+)' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group -1 in regex '(\d+)([a-z]+)' + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(?l)') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "parameter" : "`regexp`", + "value" : "'(?l)'" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 1 in regex '\d+' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 1 in regex '\d+' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 2 in regex '\d+' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group -1 in regex '\d+' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct> +-- !query output +["1a","2b","14m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct> +-- !query output +["a","b","m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group 3 in regex '(\d+)([a-z]+)' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +No group -1 in regex '(\d+)([a-z]+)' + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "parameter" : "`regexp`", + "value" : "'], ['" + } +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something') +-- !query schema +struct +-- !query output +something, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "-2", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, -2)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', -2)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, 0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', 0)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 1) +-- !query schema +struct +-- !query output +something, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 2) +-- !query schema +struct +-- !query output +hsomething, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 8) +-- !query schema +struct +-- !query output +healthy, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 26) +-- !query schema +struct +-- !query output +healthy, wealthy, and wissomething + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 27) +-- !query schema +struct +-- !query output +healthy, wealthy, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 30) +-- !query schema +struct +-- !query output +healthy, wealthy, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_like('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT regexp_like('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT regexp('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT regexp('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT rlike('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT rlike('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT regexp_count('1a 2b 14m', '\\d+') +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regexp_count('1a 2b 14m', 'mmm') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_count('the fox', 'FOX') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_count('the fox', '(?i)FOX') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT regexp_count(null, 'abc') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_count('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+ ') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)') +-- !query schema +struct +-- !query output +1a + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query schema +struct +-- !query output +14m + + +-- !query +SELECT regexp_substr('1a 2b 14m', '') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr(null, '.*') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr('abc', 'b') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT regexp_instr('abc', 'x') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_instr('ABC', '(?-i)b') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT regexp_instr('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr(null, 'b') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_instr`", + "parameter" : "`regexp`", + "value" : "') ?'" + } +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out new file mode 100644 index 000000000000..0f0f165565cc --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out @@ -0,0 +1,1738 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select concat_ws() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`concat_ws`" + } +} + + +-- !query +select format_string() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`format_string`" + } +} + + +-- !query +select 'a' || 'b' || 'c' +-- !query schema +struct +-- !query output +abc + + +-- !query +select replace('abc', 'b', '123') +-- !query schema +struct +-- !query output +a123c + + +-- !query +select replace('abc', 'b') +-- !query schema +struct +-- !query output +ac + + +-- !query +select length(uuid()), (uuid() <> uuid()) +-- !query schema +struct +-- !query output +36 true + + +-- !query +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query schema +struct +-- !query output +4 NULL NULL + + +-- !query +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query schema +struct +-- !query output +ab abcd ab NULL + + +-- !query +select left(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query schema +struct +-- !query output + NULL + + +-- !query +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query schema +struct +-- !query output +cd abcd cd NULL + + +-- !query +select right(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query schema +struct +-- !query output + NULL + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query schema +struct> +-- !query output +["aa","cc","ee",""] + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query schema +struct> +-- !query output +["aa","cc2ee3"] + + +-- !query +SELECT split('hello', '') +-- !query schema +struct> +-- !query output +["h","e","l","l","o"] + + +-- !query +SELECT split('', '') +-- !query schema +struct> +-- !query output +[""] + + +-- !query +SELECT split('abc', null) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split(null, 'b') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split_part('11.12.13', '.', 2) +-- !query schema +struct +-- !query output +12 + + +-- !query +SELECT split_part('11.12.13', '.', -1) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT split_part('11.12.13', '.', -3) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '', 1) +-- !query schema +struct +-- !query output +11.12.13 + + +-- !query +SELECT split_part('11ab12ab13', 'ab', 1) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '.', 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +SELECT split_part('11.12.13', '.', 4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', 5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', -5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part(null, '.', 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT substr('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substr('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz ") +-- !query schema +struct +-- !query output +xyz xyz xyz + + +-- !query +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query schema +struct +-- !query output +Tom Tom + + +-- !query +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query schema +struct +-- !query output +bar bar + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytest') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query schema +struct +-- !query output +testxyz + + +-- !query +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query schema +struct +-- !query output +XxyLAST WORD + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query schema +struct +-- !query output +xyztest + + +-- !query +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query schema +struct +-- !query output +TURNERyxX + + +-- !query +SELECT btrim('xyxtrimyyx', 'xy') +-- !query schema +struct +-- !query output +trim + + +-- !query +SELECT btrim(encode(" xyz ", 'utf-8')) +-- !query schema +struct +-- !query output +xyz + + +-- !query +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')) +-- !query schema +struct +-- !query output +Tom + + +-- !query +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')) +-- !query schema +struct +-- !query output +bar + + +-- !query +SELECT lpad('hi', 'invalid_length') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT rpad('hi', 'invalid_length') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT hex(lpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(lpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +000000AABB + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(lpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2EAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(rpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +AABB000000 + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT lpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +WWabc + + +-- !query +SELECT lpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +abcaW + + +-- !query +SELECT rpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +abcWW + + +-- !query +SELECT rpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +Wabca + + +-- !query +select decode() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "decode()" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "decode(encode('abc', 'utf-8'))" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8'), 'utf-8') +-- !query schema +struct +-- !query output +abc + + +-- !query +select decode(1, 1, 'Southlake') +-- !query schema +struct +-- !query output +Southlake + + +-- !query +select decode(2, 1, 'Southlake') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +San Francisco + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +Non domestic + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks') +-- !query schema +struct +-- !query output +SQL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.') +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT CONTAINS(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', 'Spark') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SQL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SPARK') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'ark') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark SQL', 'QL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(12, '1') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(true, 'ru') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'12', 12) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT contains(true, false) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(x'537061726b2053514c', x'53516c') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_number('454', '000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('454.2', '000.0') +-- !query schema +struct +-- !query output +454.2 + + +-- !query +select to_number('12,454', '00,000') +-- !query schema +struct +-- !query output +12454 + + +-- !query +select to_number('$78.12', '$00.00') +-- !query schema +struct +-- !query output +78.12 + + +-- !query +select to_number('+454', 'S000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('-454', 'S000') +-- !query schema +struct +-- !query output +-454 + + +-- !query +select to_number('12,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-12454.8 + + +-- !query +select to_number('00,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-454.8 + + +-- !query +select to_number('<00,454.8>', '00,000.9PR') +-- !query schema +struct, 00,000.9PR):decimal(6,1)> +-- !query output +-454.8 + + +-- !query +select to_binary('', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ab cd ', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab c=', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab cdef= = ', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query schema +struct +-- !query output +one two three four five six seven eight nine ten eleven twelve thirteen fourteen fivteen sixteen seventeen eightee + + +-- !query +select to_binary('a', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a?', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a?'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcde', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcde'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcd=', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcd='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a===', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a==='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('ab==f', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'ab==f'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query schema +struct +-- !query output +∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β) + + +-- !query +select to_binary('大千世界', 'utf8') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select to_binary('', 'utf-8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'utf8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('737472696E67') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('737472696E67', 'hex') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('1', 'hex') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('FF') +-- !query schema +struct +-- !query output +� + + +-- !query +select to_binary('123', 'hex') +-- !query schema +struct +-- !query output +# + + +-- !query +select to_binary('12345', 'hex') +-- !query schema +struct +-- !query output +#E + + +-- !query +select to_binary('abc', concat('utf', '-8')) +-- !query schema +struct +-- !query output +abc + + +-- !query +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abc', 'Hex') +-- !query schema +struct +-- !query output + +� + + +-- !query +select to_binary('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, 'utf-8') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, cast(null as string)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary('abc', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'1'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, 1)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_binary('abc', 1)" + } ] +} + + +-- !query +select to_binary('abc', 'invalidFormat') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'invalidformat'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, invalidFormat)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "to_binary('abc', 'invalidFormat')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT to_binary('abc', fmtField) FROM fmtTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"fmtField\"", + "inputName" : "fmt", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"to_binary(abc, fmtField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "to_binary('abc', fmtField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS fmtTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select luhn_check('4111111111111111') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('5500000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('340000000000009') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000005') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('378282246310006') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('0') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('4111111111111111 ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('4111111 111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' 4111111111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('510B105105105106') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('ABCDED') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select luhn_check(6011111111111117) +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check(6011111111111118) +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(123.456) +-- !query schema +struct +-- !query output +false diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out new file mode 100644 index 000000000000..d183e0a2cabe --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out @@ -0,0 +1,271 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view v (c) as values (1), (null) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view v_empty (e) as select 1 where false +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t(c int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t values (1), (null) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t2(d int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t2 values (2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t_empty(e int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.legacy.nullInEmptyListBehavior = false +-- !query schema +struct +-- !query output +spark.sql.legacy.nullInEmptyListBehavior false + + +-- !query +select c, c in (select e from t_empty) from t +-- !query schema +struct +-- !query output +1 false +NULL false + + +-- !query +select c, c in (select e from v_empty) from v +-- !query schema +struct +-- !query output +1 false +NULL false + + +-- !query +select c, c not in (select e from t_empty) from t +-- !query schema +struct +-- !query output +1 true +NULL true + + +-- !query +select c, c not in (select e from v_empty) from v +-- !query schema +struct +-- !query output +1 true +NULL true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding + + +-- !query +select null in (select e from t_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +false + + +-- !query +select null in (select e from v_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +false + + +-- !query +select null not in (select e from t_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +true + + +-- !query +select null not in (select e from v_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select * from t left join t2 on (t.c in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +select * from t left join t2 on (t.c not in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +set spark.sql.legacy.nullInEmptyListBehavior = true +-- !query schema +struct +-- !query output +spark.sql.legacy.nullInEmptyListBehavior true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding + + +-- !query +select null in (select e from t_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +NULL + + +-- !query +select null in (select e from v_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +NULL + + +-- !query +select null not in (select e from t_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +NULL + + +-- !query +select null not in (select e from v_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +NULL + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select * from t left join t2 on (t.c in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL 2 + + +-- !query +select * from t left join t2 on (t.c not in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL 2 + + +-- !query +reset spark.sql.legacy.nullInEmptyListBehavior +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t_empty +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 000000000000..0d5675fa6fde --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,1017 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from dummy(3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "messageParameters" : { + "name" : "`dummy`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 22, + "fragment" : "dummy(3)" + } ] +} + + +-- !query +select * from range(6 + cos(3)) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 + + +-- !query +select * from range(5, 10) +-- !query schema +struct +-- !query output +5 +6 +7 +8 +9 + + +-- !query +select * from range(0, 10, 2) +-- !query schema +struct +-- !query output +0 +2 +4 +6 +8 + + +-- !query +select * from range(0, 10, 1, 200) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +select * from range(1, 1, 1, 1, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4]", + "functionName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(1, 1, 1, 1, 1)" + } ] +} + + +-- !query +select * from range(1, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(1, null)" + } ] +} + + +-- !query +select * from range(array(1, 2, 3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"array(1, 2, 3)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 35, + "fragment" : "range(array(1, 2, 3))" + } ] +} + + +-- !query +select * from range(0, 5, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(0, 5, 0)" + } ] +} + + +-- !query +select * from RaNgE(2) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select i from range(0, 2) t(i) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select * from explode(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(array(1, 2)) t(c1) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) t(k, v) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array(rand(0))) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +select * from explode(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(null)" + } ] +} + + +-- !query +select * from explode(null) t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "explode(null) t(c1)" + } ] +} + + +-- !query +select * from explode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "explode(1)" + } ] +} + + +-- !query +select * from explode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`explode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(1, 2)" + } ] +} + + +-- !query +select * from explode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "explode(explode(array(1)))" + } ] +} + + +-- !query +select * from explode(array(1, 2)) t(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "2", + "funcName" : "`explode`", + "outColsNum" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 44, + "fragment" : "explode(array(1, 2)) t(c1, c2)" + } ] +} + + +-- !query +select * from explode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode_outer(array()) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select * from explode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from range(2) join explode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 1 +1 2 + + +-- !query +select * from range(2) join explode_outer(array()) +-- !query schema +struct +-- !query output +0 NULL +1 NULL + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output + + + +-- !query +select * from inline(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"inline(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 26, + "fragment" : "inline(null)" + } ] +} + + +-- !query +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`inline`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 66, + "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)" + } ] +} + + +-- !query +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(array(1, 2)) t(pos, x) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"posexplode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "posexplode(1)" + } ] +} + + +-- !query +select * from posexplode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`posexplode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 30, + "fragment" : "posexplode(1, 2)" + } ] +} + + +-- !query +select * from posexplode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"posexplode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 43, + "fragment" : "posexplode(explode(array(1)))" + } ] +} + + +-- !query +select * from posexplode(array(1, 2)) t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`posexplode`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "posexplode(array(1, 2)) t(x)" + } ] +} + + +-- !query +select * from posexplode_outer(array()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select * from posexplode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c') +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a') +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b') +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from json_tuple() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 39, + "fragment" : "json_tuple('{\"a\": 1}', 1)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1}', null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "json_tuple('{\"a\": 1}', null)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`json_tuple`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 62, + "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)" + } ] +} + + +-- !query +select * from stack(1, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 3 + + +-- !query +select * from stack(2, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 +3 NULL + + +-- !query +select * from stack(3, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from stack(4, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 a +2 2.2 b + + +-- !query +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 NULL +2 NULL b + + +-- !query +select * from stack() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`stack`" + } +} + + +-- !query +select * from stack(2, 1, 2, 3) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`stack`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "stack(2, 1, 2, 3) t(a, b, c)" + } ] +} + + +-- !query +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "columnIndex" : "1", + "leftParamIndex" : "2", + "leftType" : "\"STRING\"", + "rightParamIndex" : "5", + "rightType" : "\"DECIMAL(2,1)\"", + "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 50, + "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')" + } ] +} + + +-- !query +select * from stack(2, explode(array(1, 2, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"stack(2, explode(array(1, 2, 3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "stack(2, explode(array(1, 2, 3)))" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out new file mode 100644 index 000000000000..ce1df578d25d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out @@ -0,0 +1,553 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_add(2147483647, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(2147483647, decimal(1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(2147483647, "1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "1", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_add(interval 2 year, interval 2 second)" + } ] +} + + +-- !query +SELECT try_add(interval 2147483647 month, interval 2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, 0.5) +-- !query schema +struct +-- !query output +2.000000 + + +-- !query +SELECT try_divide(1, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-4.6566128730773926E-10 + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-1.0842021724855044E-19 + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, decimal(0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, "0") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(2147483647, decimal(-1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(2147483647, "-1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2, 3) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2147483647, decimal(-2)) +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(2147483647, "-2") +-- !query schema +struct +-- !query output +-4.294967294E9 + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query schema +struct +-- !query output +0-0 + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out new file mode 100644 index 000000000000..aa927cbd1174 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out @@ -0,0 +1,199 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT TRY_CAST('abc' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('abc' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST(NULL AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST(NULL AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('123.a' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('123.a' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('-2147483648' AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT TRY_CAST('-2147483649' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('2147483647' AS int) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT TRY_CAST('2147483648' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('-9223372036854775808' AS long) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT TRY_CAST('-9223372036854775809' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('9223372036854775807' AS long) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT TRY_CAST('9223372036854775808' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('interval 3 month 1 hour' AS interval) +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT TRY_CAST('abc' AS interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select TRY_CAST('true' as boolean) +-- !query schema +struct +-- !query output +true + + +-- !query +select TRY_CAST('false' as boolean) +-- !query schema +struct +-- !query output +false + + +-- !query +select TRY_CAST('abc' as boolean) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST("2021-01-01" AS date) +-- !query schema +struct +-- !query output +2021-01-01 + + +-- !query +SELECT TRY_CAST("2021-101-01" AS date) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST("2021-01-01 00:00:00" AS timestamp) +-- !query schema +struct +-- !query output +2021-01-01 00:00:00 + + +-- !query +SELECT TRY_CAST("2021-101-01 00:00:00" AS timestamp) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out new file mode 100644 index 000000000000..b70d766a6e21 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out @@ -0,0 +1,32 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select try_to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select try_to_timestamp('2016-12-31 abc') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out new file mode 100644 index 000000000000..77b6a1b14338 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +SQL array indices start at 1 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), -4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(map('a','b'), 'a') +-- !query schema +struct +-- !query output +b + + +-- !query +SELECT try_element_at(map('a','b'), 'abc') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out new file mode 100644 index 000000000000..31564008c4e1 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -0,0 +1,279 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 'aa' as a +-- !query schema +struct<> +-- !query output + + + +-- !query +select cast(a as byte) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as short) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as int) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as long) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as float) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as double) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as decimal) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as boolean) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as timestamp) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as date) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(a as binary) from t +-- !query schema +struct +-- !query output +aa + + +-- !query +select cast(a as array) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"ARRAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(a as array)" + } ] +} + + +-- !query +select cast(a as struct) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"STRUCT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(a as struct)" + } ] +} + + +-- !query +select cast(a as map) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "cast(a as map)" + } ] +} + + +-- !query +select to_timestamp(a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp(a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp(a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime(a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_unixtime('2018-01-01', a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(a, 'MO') from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day('2018-01-01', a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select trunc(a, 'MM') from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select trunc('2018-01-01', a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unhex('-123') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sha2(a, a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get_json_object(a, a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_tuple(a, a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_json(a, 'a INT') from t +-- !query schema +struct> +-- !query output +{"a":null} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out new file mode 100644 index 000000000000..d3735acf0f08 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out @@ -0,0 +1,689 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query schema +struct +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) +-- !query schema +struct<(udf((a + 1)) + 1):int,udf(count(b)):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "udf(COUNT(b))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 58, + "fragment" : "k" + } ] +} + + +-- !query +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT udf(1) FROM range(10) HAVING true +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT udf(id) FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true + + +-- !query +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query schema +struct +-- !query output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k) +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(udf(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(udf(1))" + } ] +} + + +-- !query +SELECT some(udf(1S)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "some(udf(1S))" + } ] +} + + +-- !query +SELECT any(udf(1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "any(udf(1L))" + } ] +} + + +-- !query +SELECT udf(every("true")) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 24, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out new file mode 100644 index 000000000000..d65ebb54528d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out @@ -0,0 +1,120 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST') +-- !query schema +struct +-- !query output +spark.apache.org + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH') +-- !query schema +struct +-- !query output +/path + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY') +-- !query schema +struct +-- !query output +query=1 + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF') +-- !query schema +struct +-- !query output +Ref + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL') +-- !query schema +struct +-- !query output +http + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE') +-- !query schema +struct +-- !query output +/path?query=1 + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY') +-- !query schema +struct +-- !query output +userinfo@spark.apache.org + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO') +-- !query schema +struct +-- !query output +userinfo + + +-- !query +select url_encode('https://spark.apache.org') +-- !query schema +struct +-- !query output +https%3A%2F%2Fspark.apache.org + + +-- !query +select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query schema +struct +-- !query output +inva+lid%3A%2F%2Fuser%3Apass%40host%2Ffile%5C%3Bparam%3Fquery%5C%3Bp2 + + +-- !query +select url_encode(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select url_decode('https%3A%2F%2Fspark.apache.org') +-- !query schema +struct +-- !query output +https://spark.apache.org + + +-- !query +select url_decode('http%3A%2F%2spark.apache.org') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +Illegal hex characters in escape (%) pattern: 2s + + +-- !query +select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query schema +struct +-- !query output +inva lid://user:pass@host/file\;param?query\;p2 + + +-- !query +select url_decode(null) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/window.sql.out b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/window.sql.out new file mode 100644 index 000000000000..bb6b862fc500 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/backends-velox/sql-tests/results/window.sql.out @@ -0,0 +1,1416 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_ignore_null AS SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 + + +-- !query +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "\"INT\"", + "exprType" : "\"BIGINT\"", + "location" : "upper", + "sqlExpr" : "\"ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 128, + "fragment" : "(PARTITION BY cate ORDER BY val_long\nROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 2 +1 a 2 +2 a 3 +1 b 1 +2 b 2 +3 b 2 + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 4 +1 a 4 +2 a 2 +1 b 3 +2 b 5 +3 b 3 + + +-- !query +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query schema +struct +-- !query output +NULL NULL NULL +1 NULL 1 +1 a 4 +1 a 4 +2 a 2147483652 +2147483650 a 2147483650 +NULL b NULL +3 b 2147483653 +2147483650 b 2147483650 + + +-- !query +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double +-- !query schema +struct +-- !query output +NULL NULL NULL +1.0 NULL 1.0 +1.0 a 4.5 +1.0 a 4.5 +2.5 a 2.5 +100.001 a 100.001 +1.0 b 4.3 +3.3 b 3.3 +100.001 b 100.001 + + +-- !query +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-08-01 NULL 2017-08-01 +2017-08-01 a 2017-08-02 +2017-08-01 a 2017-08-02 +2017-08-02 a 2017-08-02 +2020-12-31 a 2020-12-31 +2017-08-01 b 2017-08-03 +2017-08-03 b 2017-08-03 +2020-12-31 b 2020-12-31 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5015456E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5015456E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_date +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 160, + "fragment" : "(PARTITION BY cate ORDER BY val_date\nRANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 2 +1 a 2 +2 a 4 +1 b 1 +2 b 3 +3 b 5 + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", + "sqlState" : "42K09", + "messageParameters" : { + "lower" : "\"UNBOUNDED FOLLOWING\"", + "sqlExpr" : "\"ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING\"", + "upper" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 101, + "fragment" : "(PARTITION BY cate\nROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 94, + "fragment" : "(PARTITION BY cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpec" : "val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val ASC NULLS FIRST, cate ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 113, + "fragment" : "(PARTITION BY cate ORDER BY val, cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 121, + "fragment" : "(PARTITION BY cate ORDER BY current_timestamp\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", + "sqlState" : "42K09", + "messageParameters" : { + "comparison" : "less than or equal", + "sqlExpr" : "\"RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 107, + "fragment" : "(PARTITION BY cate ORDER BY val\nRANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 117, + "fragment" : "current_date PRECEDING" + } ] +} + + +-- !query +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +any_value(val) OVER w AS any_value, +any_value(val, true) OVER w AS any_value_ignore_null, +any_value(val, false) OVER w AS any_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct, +covar_pop(val, val_long) OVER w AS covar_pop, +corr(val, val_long) OVER w AS corr, +stddev_samp(val) OVER w AS stddev_samp, +stddev_pop(val) OVER w AS stddev_pop, +collect_list(val) OVER w AS collect_list, +collect_set(val) OVER w AS collect_set, +skewness(val_double) OVER w AS skewness, +kurtosis(val_double) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val +-- !query schema +struct,collect_set:array,skewness:double,kurtosis:double> +-- !query output +NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL +1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 +1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 +2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 +1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 +3 b 3 1 1 3 6 2.0 1.0 1 1 1 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 + + +-- !query +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL NULL +NULL a NULL +1 a NULL +1 a NULL +2 a NULL +1 b NULL +2 b NULL +3 b NULL + + +-- !query +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1037", + "messageParameters" : { + "wf" : "row_number()" + } +} + + +-- !query +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 13 1.8571428571428572 +3 NULL 13 1.8571428571428572 +NULL a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +2 a 13 1.8571428571428572 +1 b 13 1.8571428571428572 +2 b 13 1.8571428571428572 +3 b 13 1.8571428571428572 + + +-- !query +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL false true false false true false false true false +3 NULL false true false false true false false true false +NULL a false true false false true false false true false +1 a false true false false true false false true false +1 a false true false false true false false true false +2 a false true false false true false false true false +1 b false true false false true false false true false +2 b false true false false true false false true false +3 b false true false false true false false true false + + +-- !query +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query schema +struct +-- !query output +NULL 3 +a 2 +a 2 +a 4 +b 1 +b 3 +b 6 + + +-- !query +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1030" +} + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson 5186 Leslie Thompson Leslie Thompson NULL +Anthony Bow 6627 Leslie Thompson Leslie Thompson Anthony Bow +Foon Yue Tseng 6660 Leslie Thompson Leslie Thompson Anthony Bow +Gerard Hernandez 6949 Leslie Thompson Leslie Thompson Anthony Bow +Leslie Jennings 8113 Anthony Bow Anthony Bow Foon Yue Tseng +Diane Murphy 8435 Anthony Bow Anthony Bow Foon Yue Tseng +William Patterson 8870 Gerard Hernandez Gerard Hernandez Leslie Jennings +Jeff Firrelli 8992 Leslie Jennings Leslie Jennings Diane Murphy +Julie Firrelli 9181 Leslie Jennings Leslie Jennings Diane Murphy +Steve Patterson 9441 Leslie Jennings Leslie Jennings Diane Murphy +Mary Patterson 9998 Leslie Jennings Leslie Jennings Diane Murphy +Loui Bondur 10449 William Patterson William Patterson Jeff Firrelli +George Vanauf 10563 William Patterson William Patterson Jeff Firrelli +Barry Jones 10586 William Patterson William Patterson Jeff Firrelli +Pamela Castillo 11303 Steve Patterson Steve Patterson Mary Patterson +Gerard Bondur 11472 Mary Patterson Mary Patterson Loui Bondur +Larry Bott 11798 Mary Patterson Mary Patterson Loui Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Gerard Bondur Gerard Bondur Pamela Castillo +George Vanauf 10563 Pamela Castillo Pamela Castillo Barry Jones +Loui Bondur 10449 Barry Jones Barry Jones George Vanauf +Mary Patterson 9998 George Vanauf George Vanauf Loui Bondur +Steve Patterson 9441 Loui Bondur Loui Bondur Mary Patterson +Julie Firrelli 9181 Mary Patterson Mary Patterson Steve Patterson +Jeff Firrelli 8992 Steve Patterson Steve Patterson Julie Firrelli +William Patterson 8870 Julie Firrelli Julie Firrelli Jeff Firrelli +Diane Murphy 8435 Jeff Firrelli Jeff Firrelli William Patterson +Leslie Jennings 8113 William Patterson William Patterson Diane Murphy +Gerard Hernandez 6949 Diane Murphy Diane Murphy Leslie Jennings +Foon Yue Tseng 6660 Leslie Jennings Leslie Jennings Gerard Hernandez +Anthony Bow 6627 Gerard Hernandez Gerard Hernandez Foon Yue Tseng +Leslie Thompson 5186 Foon Yue Tseng Foon Yue Tseng Anthony Bow + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Gerard Bondur Gerard Bondur Pamela Castillo +Pamela Castillo 11303 Pamela Castillo Pamela Castillo Barry Jones +Barry Jones 10586 Barry Jones Barry Jones George Vanauf +George Vanauf 10563 George Vanauf George Vanauf Loui Bondur +Loui Bondur 10449 Loui Bondur Loui Bondur Mary Patterson +Mary Patterson 9998 Mary Patterson Mary Patterson Steve Patterson +Steve Patterson 9441 Steve Patterson Steve Patterson Julie Firrelli +Julie Firrelli 9181 Julie Firrelli Julie Firrelli Jeff Firrelli +Jeff Firrelli 8992 Jeff Firrelli Jeff Firrelli William Patterson +William Patterson 8870 William Patterson William Patterson Diane Murphy +Diane Murphy 8435 Diane Murphy Diane Murphy Leslie Jennings +Leslie Jennings 8113 Leslie Jennings Leslie Jennings Gerard Hernandez +Gerard Hernandez 6949 Gerard Hernandez Gerard Hernandez Foon Yue Tseng +Foon Yue Tseng 6660 Foon Yue Tseng Foon Yue Tseng Anthony Bow +Anthony Bow 6627 Anthony Bow Anthony Bow Leslie Thompson +Leslie Thompson 5186 Leslie Thompson Leslie Thompson NULL + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + department, + salary, + FIRST_VALUE(employee_name) OVER w highest_salary, + NTH_VALUE(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +) +ORDER BY department +-- !query schema +struct +-- !query output +Mary Patterson Accounting 9998 Gerard Bondur Mary Patterson +Jeff Firrelli Accounting 8992 Gerard Bondur Mary Patterson +William Patterson Accounting 8870 Gerard Bondur Mary Patterson +Diane Murphy Accounting 8435 Gerard Bondur Mary Patterson +Anthony Bow Accounting 6627 Gerard Bondur Mary Patterson +Gerard Bondur Accounting 11472 Gerard Bondur Mary Patterson +Leslie Jennings IT 8113 Leslie Jennings Leslie Thompson +Leslie Thompson IT 5186 Leslie Jennings Leslie Thompson +Larry Bott SCM 11798 Larry Bott Pamela Castillo +Gerard Hernandez SCM 6949 Larry Bott Pamela Castillo +Loui Bondur SCM 10449 Larry Bott Pamela Castillo +Barry Jones SCM 10586 Larry Bott Pamela Castillo +Pamela Castillo SCM 11303 Larry Bott Pamela Castillo +George Vanauf Sales 10563 George Vanauf Steve Patterson +Steve Patterson Sales 9441 George Vanauf Steve Patterson +Julie Firrelli Sales 9181 George Vanauf Steve Patterson +Foon Yue Tseng Sales 6660 George Vanauf Steve Patterson + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.REPETITIVE_WINDOW_DEFINITION", + "sqlState" : "42000", + "messageParameters" : { + "windowName" : "`w`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 226, + "stopIndex" : 394, + "fragment" : "WINDOW\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING)" + } ] +} + + +-- !query +SELECT + content, + id, + v, + lead(v, 0) IGNORE NULLS OVER w lead_0, + lead(v, 1) IGNORE NULLS OVER w lead_1, + lead(v, 2) IGNORE NULLS OVER w lead_2, + lead(v, 3) IGNORE NULLS OVER w lead_3, + lag(v, 0) IGNORE NULLS OVER w lag_0, + lag(v, 1) IGNORE NULLS OVER w lag_1, + lag(v, 2) IGNORE NULLS OVER w lag_2, + lag(v, 3) IGNORE NULLS OVER w lag_3, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL x y z NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +a 1 x x y z v x NULL NULL NULL x NULL NULL x x x +b 2 NULL NULL y z v NULL x NULL NULL x NULL NULL x x x +c 3 NULL NULL y z v NULL x NULL NULL x NULL NULL x x x +a 4 y y z v NULL y x NULL NULL x y NULL x x y +b 5 NULL NULL z v NULL NULL y x NULL x y NULL x x y +a 6 z z v NULL NULL z y x NULL x y z x x z +a 7 v v NULL NULL NULL v z y x x y z x x v +a 8 NULL NULL NULL NULL NULL NULL v z y x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL NULL NULL NULL NULL NULL +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x NULL NULL x x x +a 4 y x y NULL x x y +b 5 NULL x y NULL x x y +a 6 z x y z x x z +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL NULL NULL NULL NULL NULL +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x NULL NULL x x x +a 4 y x y NULL x x y +b 5 NULL x y NULL x x y +a 6 z x y z x x z +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x y NULL x x y +c 3 NULL x y NULL x x y +a 4 y y z NULL y y z +b 5 NULL y z v y y v +a 6 z y z v y y v +a 7 v z v NULL z z v +a 8 NULL z v NULL z z v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x y NULL x x y +c 3 NULL x y NULL x x y +a 4 y y z NULL y y z +b 5 NULL y z v y y v +a 6 z y z v y y v +a 7 v z v NULL z z v +a 8 NULL z v NULL z z v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL y z v y y v +c 3 NULL y z v y y v +a 4 y y z v y y v +b 5 NULL z v NULL z z v +a 6 z z v NULL z z v +a 7 v v NULL NULL v v v +a 8 NULL NULL NULL NULL NULL NULL NULL + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL x y z x x v +c 3 NULL x y z x x v +a 4 y x y z x x v +b 5 NULL x y z x x v +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL x y z x x v +c 3 NULL x y z x x v +a 4 y x y z x x v +b 5 NULL x y z x x v +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x y NULL x x y +a 4 y x y NULL x x y +b 5 NULL x y z x x z +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1004", + "messageParameters" : { + "windowName" : "w" + } +} + + +-- !query +SELECT + SUM(salary) OVER w sum_salary +FROM + basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1004", + "messageParameters" : { + "windowName" : "w" + } +} + + +-- !query +create or replace temp view t1 (p, o) as values (1, 1), (1, 1), (1, 2), (2, 1), (2, 1), (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from (select *, dense_rank() over (partition by p order by o) as rnk from t1) where rnk = 1 +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +2 1 1 +2 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 + + +-- !query +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 + + +-- !query +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 diff --git a/gluten-ut/spark40/src/test/resources/log4j2.properties b/gluten-ut/spark40/src/test/resources/log4j2.properties new file mode 100644 index 000000000000..fb1cadec5f5d --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/log4j2.properties @@ -0,0 +1,39 @@ +# +# 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. +# + +rootLogger.level = info +rootLogger.appenderRef.stdout.ref = STDOUT +rootLogger.appenderRef.file.ref = File + +#Console Appender +appender.console.type = Console +appender.console.name = STDOUT +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss.SSS} %p %c: %maxLen{%m}{512}%n%ex{8}%n +appender.console.filter.threshold.type = ThresholdFilter +appender.console.filter.threshold.level = warn + +#File Appender +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex + +appender.file.filter.threshold.type = ThresholdFilter +appender.file.filter.threshold.level = info diff --git a/gluten-ut/spark40/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/gluten-ut/spark40/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 000000000000..5e48c7d28041 --- /dev/null +++ b/gluten-ut/spark40/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,18 @@ +# +# 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. +# + +mock-maker-inline \ No newline at end of file diff --git a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala new file mode 100644 index 000000000000..937e3494d07a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala @@ -0,0 +1,276 @@ +/* + * 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.gluten.utils.clickhouse + +import org.apache.gluten.utils.SQLQueryTestSettings + +object ClickHouseSQLQueryTestSettings extends SQLQueryTestSettings { + override def getResourceFilePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources/backends-clickhouse/sql-tests" + + override def getSupportedSQLQueryTests: Set[String] = SUPPORTED_SQL_QUERY_LIST + + override def getOverwriteSQLQueryTests: Set[String] = OVERWRITE_SQL_QUERY_LIST + + override def getIgnoredSQLQueryTests: List[String] = IGNORE_SQL_QUERY_LIST + + // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list + val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "bitwise.sql", + "cast.sql", + "change-column.sql", + // CH- "charvarchar.sql", + "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", + "comments.sql", + "comparator.sql", + "count.sql", + "cross-join.sql", + "csv-functions.sql", + // CH- "cte-legacy.sql", + "cte-nested.sql", + // CH- "cte-nonlegacy.sql", + // CH- "cte.sql", + "current_database_catalog.sql", + "date.sql", + "datetime-formatting-invalid.sql", + // Velox had different handling for some illegal cases. + // "datetime-formatting-legacy.sql", + // "datetime-formatting.sql", + "datetime-legacy.sql", + "datetime-parsing-invalid.sql", + "datetime-parsing-legacy.sql", + "datetime-parsing.sql", + "datetime-special.sql", + // CH - "decimalArithmeticOperations.sql", + // "describe-part-after-analyze.sql", + "describe-query.sql", + "describe-table-after-alter-table.sql", + "describe-table-column.sql", + "describe.sql", + "except-all.sql", + "except.sql", + "extract.sql", + // CH - "group-by-filter.sql", + // CH - "group-by-ordinal.sql", + "grouping_set.sql", + "having.sql", + "ignored.sql", + "ilike-all.sql", + "ilike-any.sql", + "inline-table.sql", + "inner-join.sql", + "intersect-all.sql", + // CH - "interval.sql", + "join-empty-relation.sql", + // CH - "join-lateral.sql", + // CH - "json-functions.sql", + "like-all.sql", + "like-any.sql", + "limit.sql", + "literals.sql", + "map.sql", + // CH- "misc-functions.sql", + "natural-join.sql", + "null-handling.sql", + // CH- "null-propagation.sql", + "operators.sql", + "order-by-nulls-ordering.sql", + "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", + "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", + "query_regex_column.sql", + // CH- "random.sql", + // CH - "regexp-functions.sql", + "show-create-table.sql", + "show-tables.sql", + "show-tblproperties.sql", + "show-views.sql", + "show_columns.sql", + "sql-compatibility-functions.sql", + "string-functions.sql", + "struct.sql", + // CH - "subexp-elimination.sql", + "table-aliases.sql", + // CH -"table-valued-functions.sql", + "tablesample-negative.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-within-and-or.sql", + "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-order-by.sql", + // CH- "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", + // CH -"subquery/negative-cases/invalid-correlation.sql", + "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/subquery-in-from.sql", + "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", + "postgreSQL/aggregates_part3.sql", + "postgreSQL/aggregates_part4.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", + "postgreSQL/create_view.sql", + "postgreSQL/date.sql", + "postgreSQL/float4.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/interval.sql", + "postgreSQL/join.sql", + "postgreSQL/limit.sql", + "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", + "postgreSQL/select_having.sql", + "postgreSQL/select_implicit.sql", + "postgreSQL/strings.sql", + "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", + "postgreSQL/union.sql", + "postgreSQL/window_part1.sql", + "postgreSQL/window_part2.sql", + "postgreSQL/window_part3.sql", + "postgreSQL/window_part4.sql", + "postgreSQL/with.sql", + "datetime-special.sql", + "timestamp-ansi.sql", + "timestamp.sql", + "arrayJoin.sql", + "binaryComparison.sql", + "booleanEquality.sql", + "caseWhenCoercion.sql", + "concat.sql", + "dateTimeOperations.sql", + "decimalPrecision.sql", + "division.sql", + "elt.sql", + "ifCoercion.sql", + "implicitTypeCasts.sql", + "inConversion.sql", + "mapZipWith.sql", + "promoteStrings.sql", + "stringCastAndExpressions.sql", + "widenSetOperationTypes.sql", + "windowFrameCoercion.sql", + "timestamp-ltz.sql", + "timestamp-ntz.sql", + "timezone.sql", + // CH- "transform.sql", + "try_arithmetic.sql", + "try_cast.sql", + "udaf.sql", + "union.sql", + "using-join.sql", + "window.sql", + "udf-union.sql", + "udf-window.sql", + "ansi/cast.sql", + "ansi/decimalArithmeticOperations.sql", + "ansi/map.sql", + "ansi/datetime-parsing-invalid.sql", + "ansi/string-functions.sql", + // CH - "ansi/interval.sql", + "ansi/date.sql", + "ansi/timestamp.sql", + "ansi/try_arithmetic.sql", + "ansi/literals.sql", + "timestampNTZ/timestamp-ansi.sql", + "timestampNTZ/timestamp.sql", + "udf/udf-intersect-all.sql - Scala UDF", + "udf/udf-except-all.sql - Scala UDF", + "udf/udf-udaf.sql - Scala UDF", + "udf/udf-except.sql - Scala UDF", + "udf/udf-pivot.sql - Scala UDF", + "udf/udf-inline-table.sql - Scala UDF", + "udf/postgreSQL/udf-select_having.sql - Scala UDF", + "typeCoercion/native/decimalPrecision.sql", + "typeCoercion/native/ifCoercion.sql", + "typeCoercion/native/dateTimeOperations.sql", + "typeCoercion/native/booleanEquality.sql", + "typeCoercion/native/mapZipWith.sql", + "typeCoercion/native/caseWhenCoercion.sql", + "typeCoercion/native/widenSetOperationTypes.sql", + "typeCoercion/native/stringCastAndExpressions.sql", + "typeCoercion/native/inConversion.sql", + "typeCoercion/native/division.sql", + "typeCoercion/native/mapconcat.sql" + ) + + val OVERWRITE_SQL_QUERY_LIST: Set[String] = Set( + // The calculation formulas for corr, skewness, kurtosis, variance, and stddev in Velox differ + // slightly from those in Spark, resulting in some differences in the final results. + // Overwrite below test cases. + // -- SPARK-24369 multiple distinct aggregations having the same argument set + // -- Aggregate with nulls. + "group-by.sql", + "udf/udf-group-by.sql" + // Overwrite some results of regr_intercept, regr_r2, corr. + // CH - "linear-regression.sql" + ) + + val IGNORE_SQL_QUERY_LIST: List[String] = List( + "udf/udf-count.sql - Regular Python UDF", + "udf/udf-except.sql - Regular Python UDF", + "udf/udf-except-all.sql - Regular Python UDF", + "udf/udf-natural-join.sql - Regular Python UDF", + "udf/udf-outer-join.sql - Regular Python UDF", + "udf/udf-pivot.sql - Regular Python UDF", + "udf/udf-intersect-all.sql - Regular Python UDF", + "udf/udf-union.sql - Regular Python UDF", + "udf/udf-having.sql - Regular Python UDF", + "udf/udf-group-analytics.sql - Regular Python UDF", + "udf/udf-group-by.sql - Regular Python UDF", + // CH excludes following + "typeCoercion/native/windowFrameCoercion.sql", + "typeCoercion/native/promoteStrings.sql", + "typeCoercion/native/concat.sql", + // Moved from GlutenSQLQueryTestSuite.ignoreList + "window.sql", + "udf/udf-window.sql", + "group-by.sql", + "udf/udf-group-by.sql - Scala UDF", + "udaf/udaf-group-analytics.sql", + "udaf/udaf-group-by-ordinal.sql", + "udaf/udaf-group-by.sql" + ) +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala new file mode 100644 index 000000000000..38b23262fbc3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -0,0 +1,2209 @@ +/* + * 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.gluten.utils.clickhouse + +import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector._ +import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.clickhouse.ClickHouseAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.csv.{GlutenCSVLegacyTimeParserSuite, GlutenCSVv1Suite, GlutenCSVv2Suite} +import org.apache.spark.sql.execution.datasources.exchange.GlutenValidateRequirementsSuite +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.GlutenEnsureRequirementsSuite +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenCustomerExtensionSuite, GlutenSessionExtensionSuite} +import org.apache.spark.sql.gluten.GlutenFallbackSuite +import org.apache.spark.sql.hive.execution.GlutenHiveSQLQueryCHSuite +import org.apache.spark.sql.sources._ + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class ClickHouseTestSettings extends BackendTestSettings { + + enableSuite[ClickHouseAdaptiveQueryExecSuite] + .includeAllGlutenTests() + .includeByPrefix( + // exclude SPARK-29906 because gluten columnar operator will have different number of shuffle + "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-31658", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", + "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[FallbackStrategiesSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + .excludeCH("test ApproxCountDistinctForIntervals with large number of endpoints") + enableSuite[GlutenApproximatePercentileQuerySuite] + // requires resource files from Vanilla spark jar + .exclude("SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("SPARK-45786: Decimal multiply, divide, remainder, quot") + .excludeCH("% (Remainder)") + .excludeCH("SPARK-17617: % (Remainder) double % double on super big double") + .excludeCH("pmod") + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .exclude("column pruning - non-readable file") + enableSuite[GlutenBitmapExpressionsQuerySuite] + enableSuite[GlutenBitwiseExpressionsSuite] + enableSuite[GlutenBloomFilterAggregateQuerySuite] + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBloomFilterAggregateQuerySuiteCGOff] + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBroadcastExchangeSuite] + // TODO: fix the hanging problem in GLUTEN-8890 followup +// enableSuite[GlutenBroadcastJoinSuite] +// .includeCH("Shouldn't change broadcast join buildSide if user clearly specified") +// .includeCH("Shouldn't bias towards build right if user didn't specify") +// .includeCH("SPARK-23192: broadcast hint should be retained after using the cached data") +// .includeCH("broadcast join where streamed side's output partitioning is HashPartitioning") + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("disable bucketing when the output doesn't contain all bucketing columns") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + .includeCH("write bucketed data") + .includeCH("write bucketed data with sortBy") + .includeCH("write bucketed data without partitionBy") + .includeCH("write bucketed data without partitionBy with sortBy") + .includeCH("write bucketed data with bucketing disabled") + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenCSVLegacyTimeParserSuite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenCSVv1Suite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39469: Infer schema for columns with all dates") + .excludeCH("SPARK-40474: Infer schema for columns with a mix of dates and timestamp") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + // Flaky and already excluded in other cases + .exclude("Gluten - test for FAILFAST parsing mode") + + enableSuite[GlutenCSVv2Suite] + .exclude("Gluten - test for FAILFAST parsing mode") + // Rule org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown in batch + // Early Filter and Projection Push-Down generated an invalid plan + .exclude("SPARK-26208: write and read empty data to csv file with headers") + // file cars.csv include null string, Arrow not support to read + .exclude("old csv data source name works") + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + enableSuite[GlutenCTEHintSuite] + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenCachedTableSuite] + .exclude("A cached table preserves the partitioning and ordering of its cached SparkPlan") + .includeCH("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .includeCH("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + .excludeCH("Gluten - InMemoryRelation statistics") + enableSuite[GlutenCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeGlutenTest("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("Gluten - data type casting") + .exclude("cast string to date #2") + .exclude("casting to fixed-precision decimals") + .exclude("SPARK-28470: Cast should honor nullOnOverflow property") + .exclude("cast from array II") + .exclude("cast from map II") + .exclude("cast from struct II") + .exclude("cast from date") + .exclude("cast from timestamp II") + .exclude("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .exclude("SPARK-34727: cast from float II") + .exclude("SPARK-39749: cast Decimal to string") + .exclude("SPARK-42176: cast boolean to timestamp") + .exclude("null cast #2") + .exclude("cast from boolean to timestamp") + enableSuite[GlutenCoalesceShufflePartitionsSuite] + .excludeByPrefix("determining the number of reducers") + .excludeCH("SPARK-46590 adaptive query execution works correctly with broadcast join and union") + .excludeCH("SPARK-46590 adaptive query execution works correctly with cartesian join and union") + .excludeCH("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") + .excludeCH("Do not reduce the number of shuffle partition for repartition") + .excludeCH("Union two datasets with different pre-shuffle partition number") + .excludeCH("SPARK-34790: enable IO encryption in AQE partition coalescing") + .excludeCH("Gluten - determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: aggregate operator") + .excludeCH("Gluten - determining the number of reducers: join operator") + .excludeCH("Gluten - determining the number of reducers: complex query 1") + .excludeCH("Gluten - determining the number of reducers: complex query 2") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned") + enableSuite[GlutenCollapseProjectExecTransformerSuite] + .excludeCH("Gluten - Support ProjectExecTransformer collapse") + enableSuite[GlutenCollectionExpressionsSuite] + // Rewrite in Gluten to replace Seq with Array + .exclude("Shuffle") + .excludeGlutenTest("Shuffle") + .excludeCH("Sequence of numbers") + .excludeCH("Array Insert") + .excludeCH("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") + .excludeCH( + "SPARK-36740: ArrayMin/ArrayMax/SortArray should handle NaN greater than non-NaN value") + .excludeCH("SPARK-42401: Array insert of null value (explicit)") + .excludeCH("SPARK-42401: Array insert of null value (implicit)") + enableSuite[GlutenColumnExpressionSuite] + // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. + // The final caught Spark exception's getCause().getMessage() contains 'errMsg' but does not + // equal 'errMsg' exactly. The following two tests will be skipped and overridden in Gluten. + .includeCH("raise_error") + .includeCH("assert_true") + .excludeCH("withField should add field with no name") + .excludeCH("withField should replace all fields with given name in struct") + .excludeCH("withField user-facing examples") + .excludeCH("dropFields should drop field with no name in struct") + .excludeCH("dropFields should drop all fields with given name in struct") + enableSuite[GlutenComplexTypeSuite] + enableSuite[GlutenComplexTypesSuite] + enableSuite[GlutenConditionalExpressionSuite] + .excludeCH("case when") + enableSuite[GlutenConfigBehaviorSuite] + // Will be fixed by cleaning up ColumnarShuffleExchangeExec. + .exclude("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + // Gluten columnar operator will have different number of jobs + .exclude("SPARK-40211: customize initialNumPartitions for take") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCreateTableAsSelectSuite] + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCustomerExtensionSuite] + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenDSV2SQLInsertTestSuite] + enableSuite[GlutenDataFrameAggregateSuite] + // Test for vanilla spark codegen, not apply for Gluten + .exclude("SPARK-43876: Enable fast hashmap for distinct queries") + .exclude( + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // incorrect result, distinct NaN case + "SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", + // Replaced with another test. + "SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it" + ) + .includeCH( + "zero moments", // [velox does not return NaN] + // Velox's collect_list / collect_set are by design declarative aggregate so plan check + // for ObjectHashAggregateExec will fail. + "SPARK-22223: ObjectHashAggregate should not introduce unnecessary shuffle", + "SPARK-31620: agg with subquery (whole-stage-codegen = true)", + "SPARK-31620: agg with subquery (whole-stage-codegen = false)" + ) + .excludeCH("linear regression") + .excludeCH("collect functions") + .excludeCH("collect functions structs") + .excludeCH("SPARK-17641: collect functions should not collect null values") + .excludeCH("collect functions should be able to cast to array type with no null values") + .excludeCH("SPARK-45599: Neither 0.0 nor -0.0 should be dropped when computing percentile") + .excludeCH("SPARK-34716: Support ANSI SQL intervals by the aggregate function `sum`") + .excludeCH("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") + .excludeCH("SPARK-35412: groupBy of year-month/day-time intervals should work") + .excludeCH("SPARK-36054: Support group by TimestampNTZ column") + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + // blocked by Velox-5768 + .exclude("aggregate function - array for primitive type containing null") + .exclude("aggregate function - array for non-primitive type") + // Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown + .exclude("map_concat function") + // Rewrite this test because Velox sorts rows by key for primitive data types, which disrupts the original row sequence. + .includeCH("map_zip_with function - map of primitive types") + .excludeCH("map with arrays") + .excludeCH("flatten function") + .excludeCH("SPARK-41233: array prepend") + .excludeCH("array_insert functions") + .excludeCH("aggregate function - array for primitive type not containing null") + .excludeCH("transform keys function - primitive data types") + .excludeCH("transform values function - test primitive data types") + .excludeCH("transform values function - test empty") + .excludeCH("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") + .excludeCH("mask function") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite] + .excludeCH("SPARK-32693: Compare two dataframes with same schema except nullable property") + enableSuite[GlutenDataFrameNaFunctionsSuite] + .includeCH( + // NaN case + "replace nan with float", + "replace nan with double" + ) + enableSuite[GlutenDataFramePivotSuite] + // substring issue + .includeCH("pivot with column definition in groupby") + // array comparison not supported for values that contain nulls + .includeCH( + "pivot with null and aggregate type not supported by PivotFirst returns correct result") + .excludeCH("SPARK-38133: Grouping by TIMESTAMP_NTZ should not corrupt results") + enableSuite[GlutenDataFrameRangeSuite] + .excludeCH("SPARK-20430 Initialize Range parameters in a driver side") + .excludeByPrefix("Cancelling stage in a query with Range") + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + .excludeCH("simple session window with record at window start") + .excludeCH("session window groupBy statement") + .excludeCH("session window groupBy with multiple keys statement") + .excludeCH("session window groupBy with multiple keys statement - two distinct") + .excludeCH( + "session window groupBy with multiple keys statement - keys overlapped with sessions") + .excludeCH("SPARK-36465: filter out events with negative/zero gap duration") + .excludeCH("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH( + "SPARK-49836 using window fn with window as parameter should preserve parent operator") + enableSuite[GlutenDataFrameSetOperationsSuite] + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + .excludeCH("union should union DataFrames with UDTs (SPARK-13410)") + .excludeCH( + "SPARK-35756: unionByName support struct having same col names but different sequence") + .excludeCH("SPARK-36673: Only merge nullability for Unions of struct") + .excludeCH("SPARK-36797: Union should resolve nested columns as top-level columns") + enableSuite[GlutenDataFrameStatSuite] + .excludeCH("SPARK-30532 stat functions to understand fully-qualified column name") + .excludeCH("special crosstab elements (., '', null, ``)") + enableSuite[GlutenDataFrameSuite] + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // decimal failed ut. + "SPARK-22271: mean overflows and returns null for some decimal variables", + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + "SPARK-9083: sort with non-deterministic expressions" + ) + .includeCH( + // Mismatch when max NaN and infinite value + "NaN is greater than all other non-NaN numeric values", + "distributeBy and localSort" + ) + // test for sort node not present but gluten uses shuffle hash join + .exclude("SPARK-41048: Improve output partitioning and ordering with AQE cache") + .exclude("SPARK-28224: Aggregate sum big decimal overflow") + // Rewrite this test since it checks the physical operator which is changed in Gluten + .excludeCH("SPARK-27439: Explain result should match collected result after view change") + .excludeCH("SPARK-28067: Aggregate sum should not return wrong results for decimal overflow") + .excludeCH("SPARK-35955: Aggregate avg should not return wrong results for decimal overflow") + .excludeCH("summary") + .excludeGlutenTest( + "SPARK-27439: Explain result should match collected result after view change") + .excludeCH( + "SPARK-8608: call `show` on local DataFrame with random columns should return same value") + .excludeCH( + "SPARK-8609: local DataFrame with random columns should return same value after sort") + .excludeCH("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") + .excludeCH("Uuid expressions should produce same results at retries in the same DataFrame") + .excludeCH("Gluten - repartitionByRange") + .excludeCH("Gluten - describe") + .excludeCH("Gluten - Allow leading/trailing whitespace in string before casting") + enableSuite[GlutenDataFrameTimeWindowingSuite] + .excludeCH("simple tumbling window with record at window start") + .excludeCH("SPARK-21590: tumbling window using negative start time") + .excludeCH("tumbling window groupBy statement") + .excludeCH("tumbling window groupBy statement with startTime") + .excludeCH("SPARK-21590: tumbling window groupBy statement with negative startTime") + .excludeCH("sliding window grouping") + .excludeCH("time window joins") + .excludeCH("millisecond precision sliding windows") + enableSuite[GlutenDataFrameToSchemaSuite] + .excludeCH("struct value: compatible field nullability") + .excludeCH("map value: reorder inner fields by name") + enableSuite[GlutenDataFrameTungstenSuite] + enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .includeCH("unbounded preceding/following range between with aggregation") + .includeCH("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .excludeCH("rows between should accept int/long values as boundary") + .excludeCH("reverse preceding/following range between with aggregation") + .excludeCH( + "SPARK-41793: Incorrect result for window frames defined by a range clause on large decimals") + enableSuite[GlutenDataFrameWindowFunctionsSuite] + // does not support `spark.sql.legacy.statisticalAggregate=true` (null -> NAN) + .exclude("corr, covar_pop, stddev_pop functions in specific window") + .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") + // does not support spill + .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") + .exclude("SPARK-21258: complex object in combination with spilling") + // rewrite `WindowExec -> WindowExecTransformer` + .exclude( + "SPARK-38237: require all cluster keys for child required distribution for window query") + .excludeCH("SPARK-13860: corr, covar_pop, stddev_pop functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("SPARK-13860: covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("lead/lag with ignoreNulls") + .excludeCH("SPARK-37099: Insert window group limit node for top-k computation") + .excludeCH("Gluten - corr, covar_pop, stddev_pop functions in specific window") + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + .excludeCH("view should use captured catalog and namespace for function lookup") + .excludeCH("aggregate function: lookup int average") + .excludeCH("aggregate function: lookup long average") + .excludeCH("aggregate function: lookup double average in Java") + .excludeCH("aggregate function: lookup int average w/ expression") + .excludeCH("SPARK-35390: aggregate function w/ type coercion") + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + .excludeCH("DeleteFrom: - delete with invalid predicate") + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following tests in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + // Disable this since coalesece union clauses rule will rewrite the query. + .exclude("SPARK-44653: non-trivial DataFrame unions should not break caching") + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + // Rewrite the following two tests in GlutenDatasetSuite. + .exclude("dropDuplicates: columns with same column name") + .exclude("groupBy.as") + .exclude("dropDuplicates") + .exclude("select 2, primitive and tuple") + .exclude("SPARK-16853: select, case class and tuple") + // TODO: SPARK-16995 may dead loop!! + .exclude("SPARK-16995: flat mapping on Dataset containing a column created with lit/expr") + .exclude("SPARK-24762: typed agg on Option[Product] type") + .exclude("SPARK-40407: repartition should not result in severe data skew") + .exclude("SPARK-40660: Switch to XORShiftRandom to distribute elements") + enableSuite[GlutenDatasetUnpivotSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("Hour") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("to_timestamp exception mode") + // Replaced by a gluten test to pass timezone through config. + .exclude("from_unixtime") + .exclude("SPARK-42635: timestampadd near daylight saving transition") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") + .excludeCH("DayOfYear") + .excludeCH("Quarter") + .excludeCH("Month") + .excludeCH("Day / DayOfMonth") + .excludeCH("DayOfWeek") + .excludeCH("WeekDay") + .excludeCH("WeekOfYear") + .excludeCH("add_months") + .excludeCH("months_between") + .excludeCH("TruncDate") + .excludeCH("unsupported fmt fields for trunc/date_trunc results null") + .excludeCH("to_utc_timestamp") + .excludeCH("from_utc_timestamp") + .excludeCH("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") + .excludeCH("UNIX_SECONDS") + .excludeCH("TIMESTAMP_SECONDS") + enableSuite[GlutenDateFunctionsSuite] + // The below two are replaced by two modified versions. + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + // Unsupported datetime format: specifier X is not supported by velox. + .exclude("to_timestamp with microseconds precision") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") + // Legacy mode is not supported and velox getTimestamp function does not throw + // exception when format is "yyyy-dd-aa". + .exclude("function to_date") + .excludeCH("SPARK-30766: date_trunc of old timestamps to hours and days") + .excludeCH("SPARK-30793: truncate timestamps before the epoch to seconds and minutes") + .excludeCH("try_to_timestamp") + .excludeCH("Gluten - to_unix_timestamp") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenDecimalPrecisionSuite] + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite] + enableSuite[GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite] + // FIXME: complex type result mismatch + .includeCH("update nested struct fields") + .includeCH("update char/varchar columns") + enableSuite[GlutenDeltaBasedUpdateTableSuite] + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + .disable( + "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + .excludeGlutenTest("Subquery reuse across the whole plan") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + enableSuite[GlutenEmptyInSuite] + .excludeCH("IN with empty list") + enableSuite[GlutenEnsureRequirementsSuite] + enableSuite[GlutenExchangeSuite] + // ColumnarShuffleExchangeExec does not support doExecute() method + .exclude("shuffling UnsafeRows in exchange") + // ColumnarShuffleExchangeExec does not support SORT_BEFORE_REPARTITION + .exclude("SPARK-23207: Make repartition() generate consistent output") + // This test will re-run in GlutenExchangeSuite with shuffle partitions > 1 + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenExistenceJoinSuite] + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFallbackSuite] + .excludeCH("Gluten - test fallback event") + enableSuite[GlutenFileBasedDataSourceSuite] + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // error msg from velox is different & reader options is not supported, rewrite + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .includeCH("Return correct results when data columns overlap with partition columns") + .includeCH("Return correct results when data columns overlap with partition " + + "columns (nested data)") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support partition pruning") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("SPARK-41017: filter pushdown with nondeterministic predicates") + .excludeCH("SPARK-23072 Write and read back unicode column names - csv") + .excludeCH("Enabling/disabling ignoreMissingFiles using csv") + .excludeCH("SPARK-30362: test input metrics for DSV2") + .excludeCH("SPARK-35669: special char in CSV header with filter pushdown") + .excludeCH("Gluten - Spark native readers should respect spark.sql.caseSensitive - parquet") + .excludeCH("Gluten - SPARK-25237 compute correct input metrics in FileScanRDD") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using orc") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using parquet") + enableSuite[GlutenFileDataSourceV2FallBackSuite] + // Rewritten + .exclude("Fallback Parquet V2 to V1") + enableSuite[GlutenFileFormatWriterSuite] + // TODO: fix "empty file should be skipped while write to file" + .excludeCH("empty file should be skipped while write to file") + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenFileSourceCharVarcharTestSuite] + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in map key") + .includeCH("length check for input string values: nested in map value") + .includeCH("length check for input string values: nested in both map key and value") + .includeCH("length check for input string values: nested in array of struct") + .includeCH("length check for input string values: nested in array of array") + enableSuite[GlutenFileSourceCustomMetadataStructSuite] + enableSuite[GlutenFileSourceSQLInsertTestSuite] + .excludeCH("SPARK-33474: Support typed literals as partition spec values") + .excludeCH( + "SPARK-34556: checking duplicate static partition columns should respect case sensitive conf") + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + .excludeCH("unpartitioned table, single partition") + .excludeCH("SPARK-32019: Add spark.sql.files.minPartitionNum config") + .excludeCH( + "SPARK-32352: Partially push down support data filter if it mixed in partition filters") + .excludeCH("SPARK-44021: Test spark.sql.files.maxPartitionNum works as expected") + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenGeneratorFunctionSuite] + .exclude("SPARK-45171: Handle evaluated nondeterministic expression") + .excludeCH("single explode_outer") + .excludeCH("single posexplode_outer") + .excludeCH("explode_outer and other columns") + .excludeCH("aliased explode_outer") + .excludeCH("explode_outer on map") + .excludeCH("explode_outer on map with aliases") + .excludeCH("SPARK-40963: generator output has correct nullability") + .excludeCH("Gluten - SPARK-45171: Handle evaluated nondeterministic expression") + enableSuite[GlutenGroupBasedDeleteFromTableSuite] + enableSuite[GlutenGroupBasedMergeIntoTableSuite] + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenHashExpressionsSuite] + .excludeCH("sha2") + .excludeCH("SPARK-30633: xxHash with different type seeds") + enableSuite[GlutenHeaderCSVReadSchemaSuite] + .excludeCH("append column at the end") + .excludeCH("hide column at the end") + .excludeCH("change column type from byte to short/int/long") + .excludeCH("change column type from short to int/long") + .excludeCH("change column type from int to long") + .excludeCH("read byte, int, short, long together") + .excludeCH("change column type from float to double") + .excludeCH("read float and double together") + .excludeCH("change column type from float to decimal") + .excludeCH("change column type from double to decimal") + .excludeCH("read float, double, decimal together") + .excludeCH("read as string") + enableSuite[GlutenHigherOrderFunctionsSuite] + .excludeCH("ArraySort") + .excludeCH("ArrayAggregate") + .excludeCH("TransformKeys") + .excludeCH("TransformValues") + .excludeCH("SPARK-39419: ArraySort should throw an exception when the comparator returns null") + enableSuite[GlutenHiveSQLQueryCHSuite] + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .includeCH("Merge runtime bloom filters") + .excludeGlutenTest("GLUTEN-9849: bloom filter applied to partition filter") + enableSuite[GlutenInnerJoinSuiteForceShjOff] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInnerJoinSuiteForceShjOn] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInsertSuite] + // the native write staing dir is differnt with vanilla Spark for coustom partition paths + .exclude("SPARK-35106: Throw exception when rename custom partition paths returns false") + .exclude("Stop task set if FileAlreadyExistsException was thrown") + // Rewrite: Additional support for file scan with default values has been added in Spark-3.4. + // It appends the default value in record if it is not present while scanning. + // Velox supports default values for new records but it does not backfill the + // existing records and provides null for the existing ones. + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + .excludeCH("Gluten - insert partition table") + .excludeCH("Gluten - remove v1writes sort and project") + .excludeCH("Gluten - remove v1writes sort") + .excludeCH("Gluten - do not remove non-v1writes sort and project") + .excludeCH( + "Gluten - SPARK-35106: Throw exception when rename custom partition paths returns false") + .excludeCH( + "Gluten - Do not fallback write files if output columns contain Spark internal metadata") + .excludeCH("Gluten - Add metadata white list to allow native write files") + .excludeCH("Gluten - INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + .excludeCH( + "SPARK-45882: BroadcastHashJoinExec propagate partitioning should respect CoalescedHashPartitioning") + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .includeCH("$.store.book") + .includeCH("$") + .includeCH("$.store.book[0]") + .includeCH("$.store.book[*]") + .includeCH("$.store.book[*].category") + .includeCH("$.store.book[*].isbn") + .includeCH("$.store.book[*].reader") + .includeCH("$.store.basket[*]") + .includeCH("$.store.basket[*][0]") + .includeCH("$.store.basket[0][*]") + .includeCH("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") + .excludeCH("from_json - input=object, schema=array, output=array of single row") + .excludeCH("from_json - input=empty object, schema=array, output=array of single row with null") + .excludeCH("from_json - input=array of single object, schema=struct, output=single row") + .excludeCH("from_json - input=array, schema=struct, output=single row") + .excludeCH("from_json - input=empty array, schema=struct, output=single row with null") + .excludeCH("from_json - input=empty object, schema=struct, output=single row with null") + .excludeCH("SPARK-20549: from_json bad UTF-8") + .excludeCH("from_json with timestamp") + .excludeCH("to_json - struct") + .excludeCH("to_json - array") + .excludeCH("to_json - array with single empty row") + .excludeCH("to_json with timestamp") + .excludeCH("SPARK-21513: to_json support map[string, struct] to json") + .excludeCH("SPARK-21513: to_json support map[struct, struct] to json") + .excludeCH("parse date with locale") + .excludeCH("parse decimals using locale") + enableSuite[GlutenJsonFunctionsSuite] + // * in get_json_object expression not supported in velox + .exclude("SPARK-42782: Hive compatibility check for get_json_object") + // Velox does not support single quotes in get_json_object function. + .includeCH("function get_json_object - support single quotes") + .excludeCH("from_json with option (allowComments)") + .excludeCH("from_json with option (allowUnquotedFieldNames)") + .excludeCH("from_json with option (allowSingleQuotes)") + .excludeCH("from_json with option (allowNumericLeadingZeros)") + .excludeCH("from_json with option (allowBackslashEscapingAnyCharacter)") + .excludeCH("from_json with option (dateFormat)") + .excludeCH("from_json with option (allowUnquotedControlChars)") + .excludeCH("from_json with option (allowNonNumericNumbers)") + .excludeCH("from_json missing columns") + .excludeCH("from_json invalid json") + .excludeCH("from_json array support") + .excludeCH("to_json with option (timestampFormat)") + .excludeCH("to_json with option (dateFormat)") + .excludeCH("SPARK-19637 Support to_json in SQL") + .excludeCH("pretty print - roundtrip from_json -> to_json") + .excludeCH("from_json invalid json - check modes") + .excludeCH("SPARK-36069: from_json invalid json schema - check field name and field value") + .excludeCH("corrupt record column in the middle") + .excludeCH("parse timestamps with locale") + .excludeCH("SPARK-33134: return partial results only for root JSON objects") + .excludeCH("SPARK-40646: return partial results for JSON arrays with objects") + .excludeCH("SPARK-40646: return partial results for JSON maps") + .excludeCH("SPARK-40646: return partial results for objects with values as JSON arrays") + .excludeCH("SPARK-48863: parse object as an array with partial results enabled") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetStructField") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetArrayStructFields") + .excludeCH("SPARK-33907: json pruning optimization with corrupt record field") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonV2Suite] + // exception test + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenKeyGroupedPartitioningSuite] + // NEW SUITE: disable as they check vanilla spark plan + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + // disable due to check for SMJ node + .excludeByPrefix("SPARK-41413: partitioned join:") + .excludeByPrefix("SPARK-42038: partially clustered:") + .exclude("SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH("Gluten - partitioned join: only one side reports partitioning") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from one side are subset of those from the other side") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from both sides overlaps") + .excludeCH( + "Gluten - SPARK-41413: partitioned join: non-overlapping partition values from both sides") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and both sides partially clustered") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on left-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on right-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: left outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: right outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: full outer join is not applicable") + .excludeCH("Gluten - SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH( + "Gluten - partitioned join: join with two partition keys and different # of partition keys") + enableSuite[GlutenLateralColumnAliasSuite] + .excludeCH("Lateral alias conflicts with table column - Project") + .excludeCH("Lateral alias conflicts with table column - Aggregate") + .excludeCH("Lateral alias of a complex type") + .excludeCH("Lateral alias reference works with having and order by") + .excludeCH("Lateral alias basics - Window on Project") + .excludeCH("Lateral alias basics - Window on Aggregate") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("default") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenLocalBroadcastExchangeSuite] + .excludeCH("SPARK-39983 - Broadcasted relation is not cached on the driver") + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenMathExpressionsSuite] + // Spark round UT for round(3.1415,3) is not correct. + .exclude("round/bround/floor/ceil") + .excludeCH("tanh") + .excludeCH("unhex") + .excludeCH("atan2") + .excludeCH("SPARK-42045: integer overflow in round/bround") + .excludeCH("Gluten - round/bround/floor/ceil") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("read byte, int, short, long together") + .includeCH("change column type from float to double") + .includeCH("read float and double together") + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMetadataColumnSuite] + .excludeCH("SPARK-34923: propagate metadata columns through Sort") + .excludeCH("SPARK-34923: propagate metadata columns through RepartitionBy") + .excludeCH("SPARK-40149: select outer join metadata columns with DataFrame API") + .excludeCH("SPARK-42683: Project a metadata column by its logical name - column not found") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + enableSuite[GlutenOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("hide column in the middle") + .includeCH("change column position") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("read as string") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + .exclude("change column type from float to decimal") + .exclude("change column type from double to decimal") + .exclude("read float, double, decimal together") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + enableSuite[GlutenOrcSourceSuite] + // Rewrite to disable Spark's columnar reader. + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") + .includeCH("SPARK-34862: Support ORC vectorized reader for nested column") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("create temporary orc table") + .includeCH("create temporary orc table as") + .includeCH("appending insert") + .includeCH("overwrite insert") + .includeCH("SPARK-34897: Support reconcile schemas based on index after nested column pruning") + .excludeGlutenTest("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .excludeGlutenTest("SPARK-31238, SPARK-31423: rebasing dates in write") + .excludeGlutenTest("SPARK-34862: Support ORC vectorized reader for nested column") + // exclude as struct not supported + .includeCH("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .includeCH("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("Gluten - SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .excludeCH("Gluten - SPARK-31284, SPARK-31423: rebasing timestamps in write") + .excludeCH("Gluten - SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1AggregatePushDownSuite] + .includeCH("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + .includeCH("read partitioned table - partition key included in orc file") + .includeCH("read partitioned table - with nulls and partition keys are included in Orc file") + enableSuite[GlutenOrcV1QuerySuite] + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") + enableSuite[GlutenOrcV1SchemaPruningSuite] + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + // Vectorized reading. + .includeCH("Spark vectorized reader - without partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - with partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOrcV2AggregatePushDownSuite] + .includeCH("nested column: Max(top level column) not push down") + .includeCH("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV2QuerySuite] + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH( + "SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") + enableSuite[GlutenOrcV2SchemaPruningSuite] + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOuterJoinSuiteForceShjOff] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-32717: AQEOptimizer should respect excludedRules configuration") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen off)") + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen on)") + enableSuite[GlutenOuterJoinSuiteForceShjOn] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen off)") + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen on)") + enableSuite[GlutenParametersSuite] + enableSuite[GlutenParquetCodecSuite] + // codec not supported in native + .includeCH("write and read - file source parquet - codec: lz4_raw") + .includeCH("write and read - file source parquet - codec: lz4raw") + enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") + .excludeCH("test reading unaligned pages - test all types (dict encode)") + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite] + // Velox does not support rle encoding, but it can pass when native writer enabled. + .includeCH("parquet v2 pages - rle encoding for boolean value columns") + .excludeCH("All Types Dictionary") + .excludeCH("All Types Null") + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + .excludeCH( + "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") + enableSuite[GlutenParquetFileFormatV2Suite] + .excludeCH( + "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] + enableSuite[GlutenParquetIOSuite] + // Velox doesn't write file metadata into parquet file. + .includeCH("Write Spark version into Parquet metadata") + // Exception. + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. + .exclude("SPARK-35640: int as long should throw schema incompatible error") + // Velox parquet reader not allow offset zero. + .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetProtobufCompatibilitySuite] + .excludeCH("struct with unannotated array") + enableSuite[GlutenParquetReadSchemaSuite] + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + .excludeCH("Gluten - SPARK-31159: rebasing dates in write") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + // [PATH_NOT_FOUND] Path does not exist: + // file:/opt/spark331/sql/core/src/test/resources/test-data/timestamp-nanos.parquet + // May require for newer spark.test.home + .excludeByPrefix("SPARK-40819") + enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. + .exclude("Read Parquet file generated by parquet-thrift") + .excludeCH("SPARK-10136 list of primitive list") + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV1FilterSuite] + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + // avoid Velox compile error + enableSuite( + "org.apache.gluten.execution.parquet.GlutenParquetV1FilterSuite2" + ) + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV1QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .includeCH("SPARK-39833: pushed filters with project without filter columns") + .includeCH("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV1SchemaPruningSuite] + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .includeCH("aggregate push down - different data types") + enableSuite[GlutenParquetV2FilterSuite] + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + .excludeCH("Gluten - filter pushdown - date") + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV2QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV2SchemaPruningSuite] + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenPartitionedWriteSuite] + .excludeCH("SPARK-37231, SPARK-37240: Dynamic writes/reads of ANSI interval partitions") + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPredicateSuite] + .excludeCH("basic IN/INSET predicate test") + .excludeCH("IN with different types") + .excludeCH("IN/INSET: binary") + .excludeCH("IN/INSET: struct") + .excludeCH("IN/INSET: array") + .excludeCH("BinaryComparison: lessThan") + .excludeCH("BinaryComparison: LessThanOrEqual") + .excludeCH("BinaryComparison: GreaterThan") + .excludeCH("BinaryComparison: GreaterThanOrEqual") + .excludeCH("EqualTo on complex type") + .excludeCH("SPARK-32764: compare special double/float values") + .excludeCH("SPARK-32110: compare special double/float values in struct") + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + .excludeCH("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") + .excludeCH( + "ALTER NAMESPACE with property other than COMMENT for JDBC catalog should throw an exception") + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + // Doesn't support unhex with failOnError=true. + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function hex") + .excludeCH("CONVERSION_INVALID_INPUT: to_binary conversion function base64") + .excludeCH("UNSUPPORTED_FEATURE - SPARK-38504: can't read TimestampNTZ as TimestampLTZ") + .excludeCH("CANNOT_PARSE_DECIMAL: unparseable decimal") + .excludeCH("UNRECOGNIZED_SQL_TYPE: unrecognized SQL type DATALINK") + .excludeCH("UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER: The target JDBC server hosting table does not support ALTER TABLE with multiple actions.") + .excludeCH("INVALID_BITMAP_POSITION: position out of bounds") + .excludeCH("INVALID_BITMAP_POSITION: negative position") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + .excludeCH("LIKE Pattern") + .excludeCH("LIKE Pattern ESCAPE '/'") + .excludeCH("LIKE Pattern ESCAPE '#'") + .excludeCH("LIKE Pattern ESCAPE '\"'") + .excludeCH("RLIKE Regular Expression") + .excludeCH("RegexReplace") + .excludeCH("RegexExtract") + .excludeCH("RegexExtractAll") + .excludeCH("SPLIT") + enableSuite[GlutenRemoveRedundantWindowGroupLimitsSuite] + .excludeCH("remove redundant WindowGroupLimits") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .exclude("do not replace hash aggregate if child does not have sort order") + .exclude("do not replace hash aggregate if there is no group-by column") + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenResolveDefaultColumnsSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenRuntimeNullChecksV2Writes] + enableSuite[GlutenSQLAggregateFunctionSuite] + .excludeGlutenTest("Return NaN or null when dividing by zero") + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .includeCH("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .includeCH("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .includeCH("SPARK-28156: self-join should not miss cached view") + .includeCH("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .includeCH("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Not useful and time consuming. + .includeCH("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + // exception test, rewritten in gluten + .exclude("the escape character is not allowed to end with") + // ORC related + .includeCH("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .includeCH("SPARK-38173: Quoted column cannot be recognized correctly when quotedRegexColumnNames is true") + // Need to support MAP + .exclude( + "SPARK-27619: When spark.sql.legacy.allowHashOnMapType is true, hash can be used on Maptype") + .excludeCH("SPARK-6743: no columns from cache") + .excludeCH("external sorting updates peak execution memory") + .excludeCH("Struct Star Expansion") + .excludeCH("Common subexpression elimination") + .excludeCH("SPARK-24940: coalesce and repartition hint") + .excludeCH("normalize special floating numbers in subquery") + .excludeCH("SPARK-38548: try_sum should return null if overflow happens before merging") + .excludeCH("SPARK-38589: try_avg should return null if overflow happens before merging") + .excludeCH("Gluten - SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + .excludeCH("Gluten - the escape character is not allowed to end with") + // enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenSQLWindowFunctionSuite] + // spill not supported yet. + .exclude("test with low buffer spill threshold") + .excludeCH( + "window function: multiple window expressions specified by range in a single expression") + .excludeCH("Gluten - Filter on row number") + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenSessionExtensionSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + // TODO: fix the hanging problem in GLUTEN-8890 followup +// enableSuite[GlutenSortShuffleSuite] + enableSuite[GlutenSortSuite] + .excludeCH("basic sorting using ExternalSort") + .excludeCH("SPARK-33260: sort order is a Stream") + .excludeCH("SPARK-40089: decimal values sort correctly") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + enableSuite[GlutenSparkSessionExtensionSuite] + enableSuite[GlutenStatisticsCollectionSuite] + // The output byte size of Velox is different + .includeCH("SPARK-33687: analyze all tables in a specific database") + .excludeCH("analyze empty table") + .excludeCH("analyze column command - result verification") + .excludeCH("column stats collection for null columns") + .excludeCH("store and retrieve column stats in different time zones") + .excludeCH("SPARK-42777: describe column stats (min, max) for timestamp_ntz column") + .excludeCH("Gluten - store and retrieve column stats in different time zones") + .excludeCH("statistics collection of a table with zero column") + enableSuite[GlutenStringExpressionsSuite] + .excludeCH("StringComparison") + .excludeCH("Substring") + .excludeCH("string substring_index function") + .excludeCH("SPARK-40213: ascii for Latin-1 Supplement characters") + .excludeCH("ascii for string") + .excludeCH("Mask") + .excludeCH("SPARK-42384: Mask with null input") + .excludeCH("base64/unbase64 for string") + .excludeCH("encode/decode for string") + .excludeCH("SPARK-47307: base64 encoding without chunking") + .excludeCH("Levenshtein distance threshold") + .excludeCH("soundex unit test") + .excludeCH("overlay for string") + .excludeCH("overlay for byte array") + .excludeCH("translate") + .excludeCH("FORMAT") + .excludeCH("LOCATE") + .excludeCH("REPEAT") + .excludeCH("ParseUrl") + .excludeCH("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") + enableSuite[GlutenStringFunctionsSuite] + .excludeCH("string Levenshtein distance") + .excludeCH("string regexp_count") + .excludeCH("string regex_replace / regex_extract") + .excludeCH("string regexp_extract_all") + .excludeCH("string regexp_substr") + .excludeCH("string overlay function") + .excludeCH("binary overlay function") + .excludeCH("string / binary length function") + .excludeCH("SPARK-36751: add octet length api for scala") + .excludeCH("SPARK-36751: add bit length api for scala") + .excludeCH("str_to_map function") + .excludeCH("SPARK-42384: mask with null input") + .excludeCH("like & ilike function") + .excludeCH("parse_url") + .excludeCH("url_decode") + .excludeCH("url_encode") + enableSuite[GlutenSubqueryHintPropagationSuite] + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + .excludeCH("SPARK-39355: Single column uses quoted to construct UnresolvedAttribute") + .excludeCH("SPARK-40800: always inline expressions in OptimizeOneRowRelationSubquery") + .excludeCH("SPARK-40862: correlated one-row subquery with non-deterministic expressions") + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenTableLocationSuite] + enableSuite[GlutenTableOptionsConstantFoldingSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenTakeOrderedAndProjectSuite] + .excludeCH("TakeOrderedAndProject.doExecute without project") + .excludeCH("TakeOrderedAndProject.doExecute with project") + .excludeCH("TakeOrderedAndProject.doExecute with local sort") + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenTryCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + .exclude("ANSI mode: Throw exception on casting out-of-range value to byte type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to short type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to int type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to long type") + .exclude("cast from invalid string to numeric should throw NumberFormatException") + .exclude("SPARK-26218: Fix the corner case of codegen when casting float to Integer") + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeGlutenTest("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("cast from timestamp II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .excludeCH("cast from array II") + .excludeCH("cast from array III") + .excludeCH("cast from struct III") + .excludeCH("ANSI mode: cast string to timestamp with parse error") + .excludeCH("ANSI mode: cast string to date with parse error") + .excludeCH("Gluten - data type casting") + enableSuite[GlutenTryEvalSuite] + .excludeCH("try_subtract") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + // Rewrite with NaN test cases excluded. + .exclude("cases when literal is max") + enableSuite[GlutenUrlFunctionsSuite] + .excludeCH("url encode/decode function") + enableSuite[GlutenV1WriteCommandSuite] + // Rewrite to match SortExecTransformer. + .excludeByPrefix("SPARK-41914:") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition column") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - string partition column") + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + // Rewrite to disable Spark's vectorized reading. + .includeCH("change column position") + .includeCH("read byte, int, short, long together") + .includeCH("read float and double together") + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("change column type from float to double") + // .excludeGlutenTest("read byte, int, short, long together") + // .excludeGlutenTest("read float and double together") + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenXPathFunctionsSuite] + + override def getSQLQueryTestSettings: SQLQueryTestSettings = ClickHouseSQLQueryTestSettings +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala new file mode 100644 index 000000000000..180009d96df0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala @@ -0,0 +1,284 @@ +/* + * 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.gluten.utils.velox + +import org.apache.gluten.utils.SQLQueryTestSettings + +object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { + override def getResourceFilePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources/backends-velox/sql-tests" + + override def getSupportedSQLQueryTests: Set[String] = SUPPORTED_SQL_QUERY_LIST + + override def getOverwriteSQLQueryTests: Set[String] = OVERWRITE_SQL_QUERY_LIST + + // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list + // Gluten currently only supports `SET spark.sql.legacy.timeParserPolicy=LEGACY` + // Queries in `date.sql` and `timestamp.sql` are tested in `datetime-legacy.sql`. + val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "ansi/conditional-functions.sql", + "ansi/decimalArithmeticOperations.sql", + "cast.sql", + "change-column.sql", + "ceil-floor-with-scale-param.sql", + "column-resolution-aggregate.sql", + "column-resolution-sort.sql", + "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", + "comments.sql", + "comparator.sql", + "cross-join.sql", + "csv-functions.sql", + "cte-legacy.sql", + "cte-nested.sql", + "cte-nonlegacy.sql", + "current_database_catalog.sql", + // "datetime-formatting-invalid.sql", + "datetime-special.sql", + "decimalArithmeticOperations.sql", + "describe.sql", + "describe-part-after-analyze.sql", + "describe-table-after-alter-table.sql", + "describe-query.sql", + "double-quoted-identifiers.sql", + "except.sql", + "except-all.sql", + "extract.sql", + "group-by.sql", + "group-by-all.sql", + "group-by-all-duckdb.sql", + "group-by-all-mosha.sql", + "group-analytics.sql", + "group-by-filter.sql", + "group-by-ordinal.sql", + "grouping_set.sql", + "having.sql", + "higher-order-functions.sql", + "identifier-clause.sql", + "ignored.sql", + "ilike.sql", + "ilike-all.sql", + "ilike-any.sql", + "inline-table.sql", + "inner-join.sql", + "intersect-all.sql", + "join-empty-relation.sql", + "join-lateral.sql", + "json-functions.sql", + "keywords.sql", + "like-all.sql", + "like-any.sql", + "limit.sql", + "literals.sql", + "map.sql", + "mask-functions.sql", + "math.sql", + "named-function-arguments.sql", + "natural-join.sql", + "non-excludable-rule.sql", + "null-handling.sql", + "null-propagation.sql", + "operators.sql", + "order-by-all.sql", + "order-by-nulls-ordering.sql", + "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", + "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", + "query_regex_column.sql", + "random.sql", + "show-create-table.sql", + "show-tables.sql", + "show-tblproperties.sql", + "show-views.sql", + "show_columns.sql", + "sql-compatibility-functions.sql", + "struct.sql", + "subexp-elimination.sql", + "table-aliases.sql", + "table-valued-functions.sql", + "tablesample-negative.sql", + "try-string-functions.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-outside-filter.sql", + "subquery/exists-subquery/exists-within-and-or.sql", + "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-nullability.sql", + "subquery/in-subquery/in-order-by.sql", + "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", + "subquery/negative-cases/invalid-correlation.sql", + "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-count-bug.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/scalar-subquery/scalar-subquery-set-op.sql", + "subquery/subquery-in-from.sql", + "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", + "postgreSQL/aggregates_part3.sql", + "postgreSQL/aggregates_part4.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", + "postgreSQL/create_view.sql", + "postgreSQL/date.sql", + "postgreSQL/float4.sql", + "postgreSQL/float8.sql", + "postgreSQL/groupingsets.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/interval.sql", + "postgreSQL/join.sql", + "postgreSQL/limit.sql", + "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", + "postgreSQL/select_having.sql", + "postgreSQL/select_implicit.sql", + "postgreSQL/strings.sql", + "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", + "postgreSQL/union.sql", + "postgreSQL/window_part2.sql", + "postgreSQL/with.sql", + "datetime-special.sql", + "timestamp-ltz.sql", + "timestamp-ntz.sql", + "timezone.sql", + "transform.sql", + "try-string-functions.sql", + "try_aggregates.sql", + "typeCoercion/native/arrayJoin.sql", + "typeCoercion/native/binaryComparison.sql", + "typeCoercion/native/booleanEquality.sql", + "typeCoercion/native/caseWhenCoercion.sql", + "typeCoercion/native/concat.sql", + "typeCoercion/native/dateTimeOperations.sql", + "typeCoercion/native/decimalPrecision.sql", + "typeCoercion/native/division.sql", + "typeCoercion/native/elt.sql", + "typeCoercion/native/ifCoercion.sql", + // "typeCoercion/native/implicitTypeCasts.sql", + "typeCoercion/native/inConversion.sql", + "typeCoercion/native/mapconcat.sql", + "typeCoercion/native/mapZipWith.sql", + "typeCoercion/native/promoteStrings.sql", + "typeCoercion/native/widenSetOperationTypes.sql", + "typeCoercion/native/windowFrameCoercion.sql", + "udaf/udaf.sql - Grouped Aggregate Pandas UDF", + "udf/udf-union.sql - Scala UDF", + "udf/udf-intersect-all.sql - Scala UDF", + "udf/udf-except-all.sql - Scala UDF", + "udf/udf-udaf.sql - Scala UDF", + "udf/udf-except.sql - Scala UDF", + "udf/udf-pivot.sql - Scala UDF", + "udf/udf-inline-table.sql - Scala UDF", + "udf/postgreSQL/udf-select_having.sql - Scala UDF", + "union.sql", + "unpivot.sql", + "using-join.sql" + ) + + val OVERWRITE_SQL_QUERY_LIST: Set[String] = Set( + // The calculation formulas for corr, skewness, kurtosis, variance, and stddev in Velox differ + // slightly from those in Spark, resulting in some differences in the final results. + // Overwrite below test cases. + // -- SPARK-24369 multiple distinct aggregations having the same argument set + // -- Aggregate with nulls. + "group-by.sql", + "udf/udf-group-by.sql", + // Overwrite some results of regr_intercept, regr_r2, corr. + "linear-regression.sql", + // Overwrite exception message. + "array.sql", + // Overwrite exception message. + "bitwise.sql", + // Enable NullPropagation rule for + // "legacy behavior: allow calling function count without parameters". + "count.sql", + // Enable ConstantFolding rule for "typeof(...)". + "charvarchar.sql", + // Enable ConstantFolding rule for "typeof(...)". + "cte.sql", + // Removed some result mismatch cases. + "datetime-legacy.sql", + // Removed some result mismatch cases. + "datetime-parsing.sql", + // Removed some result mismatch cases. + "datetime-parsing-legacy.sql", + // Removed some result mismatch cases. + "datetime-parsing-invalid.sql", + // Overwrite exception message. See Spark-46550. + "hll.sql", + // Overwrite exception message. + "interval.sql", + // Enable ConstantFolding rule for "typeof(...)". + "misc-functions.sql", + // Removed some result mismatch cases. + "regexp-functions.sql", + // Removed some result mismatch cases. + "string-functions.sql", + // Removed some result mismatch cases. + "try_arithmetic.sql", + // Removed some result mismatch cases. + "try_cast.sql", + // Removed SQLs that can only pass with `set spark.sql.legacy.timeParserPolicy=LEGACY;` + "typeCoercion/native/stringCastAndExpressions.sql", + // Enable ConstantFolding rule for some queries. + "percentiles.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part1.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part3.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part4.sql", + // Enable NullPropagation rule for some queries that rely on the rule. + "subquery/in-subquery/in-null-semantics.sql", + // Removed some result mismatch cases. + "try_datetime_functions.sql", + // Overwrite exception message. + "try_element_at.sql", + // Overwrite exception message. + "url-functions.sql", + // Removed failed query. Adjust the output order for some queries. + "window.sql" + ) +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala new file mode 100644 index 000000000000..738f14cf0418 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -0,0 +1,1054 @@ +/* + * 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.gluten.utils.velox + +import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} + +import org.apache.spark.GlutenSortShuffleSuite +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector._ +import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.velox.VeloxAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.exchange.GlutenValidateRequirementsSuite +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.GlutenEnsureRequirementsSuite +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.python._ +import org.apache.spark.sql.extension.{GlutenSessionExtensionSuite, TestFileSourceScanExecTransformer} +import org.apache.spark.sql.gluten.GlutenFallbackSuite +import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.sources._ + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class VeloxTestSettings extends BackendTestSettings { + enableSuite[GlutenStringFunctionsSuite] + enableSuite[GlutenBloomFilterAggregateQuerySuite] + enableSuite[GlutenBloomFilterAggregateQuerySuiteCGOff] + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following tests in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenFileDataSourceV2FallBackSuite] + // Rewritten + .exclude("Fallback Parquet V2 to V1") + // TODO: fix in Spark-4.0 + // enableSuite[GlutenKeyGroupedPartitioningSuite] + // // NEW SUITE: disable as they check vanilla spark plan + // .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + // .exclude("partitioned join: only one side reports partitioning") + // .exclude("partitioned join: join with two partition keys and different # of partition keys") + // // disable due to check for SMJ node + // .excludeByPrefix("SPARK-41413: partitioned join:") + // .excludeByPrefix("SPARK-42038: partially clustered:") + // .exclude("SPARK-44641: duplicated records when SPJ is not triggered") + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenMetadataColumnSuite] + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + // Doesn't support unhex with failOnError=true. + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function hex") + // TODO: fix in Spark-4.0 + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function base64") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("SPARK-45786: Decimal multiply, divide, remainder, quot") + enableSuite[GlutenBitwiseExpressionsSuite] + enableSuite[GlutenCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Set timezone through config. + .exclude("data type casting") + // Revised by setting timezone through config and commented unsupported cases. + .exclude("cast string to timestamp") + .exclude("cast from timestamp II") + .exclude("SPARK-36286: invalid string cast to timestamp") + .exclude("SPARK-39749: cast Decimal to string") + // TODO: fix in Spark-4.0 + .exclude("Casting to char/varchar") + enableSuite[GlutenTryCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + .exclude("cast from timestamp II") // Rewrite test for Gluten not supported with ANSI mode + .exclude("ANSI mode: Throw exception on casting out-of-range value to byte type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to short type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to int type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to long type") + .exclude("cast from invalid string to numeric should throw NumberFormatException") + .exclude("SPARK-26218: Fix the corner case of codegen when casting float to Integer") + // Set timezone through config. + .exclude("data type casting") + // Revised by setting timezone through config and commented unsupported cases. + .exclude("cast string to timestamp") + // TODO: fix in Spark-4.0 + .exclude("cast from array III") + .exclude("cast from struct III") + .exclude("Casting to char/varchar") + enableSuite[GlutenCollectionExpressionsSuite] + // Rewrite in Gluten to replace Seq with Array + .exclude("Shuffle") + .excludeGlutenTest("Shuffle") + enableSuite[GlutenConditionalExpressionSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("Hour") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("to_timestamp exception mode") + // Replaced by a gluten test to pass timezone through config. + .exclude("from_unixtime") + // Vanilla Spark does not have a unified DST Timestamp fastTime. 1320570000000L and + // 1320566400000L both represent 2011-11-06 01:00:00. + .exclude("SPARK-42635: timestampadd near daylight saving transition") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") + // Replaced by a gluten test to pass timezone through config. + .exclude("months_between") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenDecimalPrecisionSuite] + enableSuite[GlutenGeneratorExpressionSuite] + enableSuite[GlutenHashExpressionsSuite] + enableSuite[GlutenHigherOrderFunctionsSuite] + enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/10948 + .exclude("$['key with spaces']") + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") + enableSuite[GlutenJsonFunctionsSuite] + // * in get_json_object expression not supported in velox + .exclude("SPARK-42782: Hive compatibility check for get_json_object") + // Velox does not support single quotes in get_json_object function. + .exclude("function get_json_object - support single quotes") + // TODO: fix in Spark-4.0 + .exclude("function get_json_object - path is null") + .exclude("function get_json_object - json is null") + .exclude("function get_json_object - Codegen Support") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("default") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenMathExpressionsSuite] + // Spark round UT for round(3.1415,3) is not correct. + .exclude("round/bround/floor/ceil") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenPredicateSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + enableSuite[GlutenSortShuffleSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + enableSuite[GlutenStringExpressionsSuite] + enableSuite[GlutenTryEvalSuite] + enableSuite[VeloxAdaptiveQueryExecSuite] + .includeAllGlutenTests() + .includeByPrefix( + "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-31658", + "SPARK-32717", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", + "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .exclude("column pruning - non-readable file") + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCSVv1Suite] + // // file cars.csv include null string, Arrow not support to read + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + // // Flaky and already excluded in other cases + // .exclude("Gluten - test for FAILFAST parsing mode") + + // enableSuite[GlutenCSVv2Suite] + // .exclude("Gluten - test for FAILFAST parsing mode") + // // Rule org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown in batch + // // Early Filter and Projection Push-Down generated an invalid plan + // .exclude("SPARK-26208: write and read empty data to csv file with headers") + // // file cars.csv include null string, Arrow not support to read + // .exclude("old csv data source name works") + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + + // enableSuite[GlutenCSVLegacyTimeParserSuite] + // // file cars.csv include null string, Arrow not support to read + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonV2Suite] + // exception test + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + .exclude("read partitioned table - partition key included in orc file") + .exclude("read partitioned table - with nulls and partition keys are included in Orc file") + enableSuite[GlutenOrcV1QuerySuite] + // Rewrite to disable Spark's columnar reader. + .exclude("Simple selection form ORC table") + .exclude("simple select queries") + .exclude("overwriting") + .exclude("self-join") + .exclude("columns only referenced by pushed down filters should remain") + .exclude("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + enableSuite[GlutenOrcV2QuerySuite] + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .exclude("Simple selection form ORC table") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .exclude("simple select queries") + .exclude("overwriting") + .exclude("self-join") + .exclude("columns only referenced by pushed down filters should remain") + .exclude("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + enableSuite[GlutenOrcSourceSuite] + // Rewrite to disable Spark's columnar reader. + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("create temporary orc table") + .exclude("create temporary orc table as") + .exclude("appending insert") + .exclude("overwrite insert") + .exclude("SPARK-34897: Support reconcile schemas based on index after nested column pruning") + .excludeGlutenTest("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .excludeGlutenTest("SPARK-31238, SPARK-31423: rebasing dates in write") + .excludeGlutenTest("SPARK-34862: Support ORC vectorized reader for nested column") + // exclude as struct not supported + .exclude("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .exclude("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1SchemaPruningSuite] + enableSuite[GlutenOrcV2SchemaPruningSuite] + enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") + // Rewrite by converting smaller integral value to timestamp. + .exclude("test reading unaligned pages - test all types (dict encode)") + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite] + // Velox does not support rle encoding, but it can pass when native writer enabled. + .exclude("parquet v2 pages - rle encoding for boolean value columns") + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + enableSuite[GlutenParquetFileFormatV2Suite] + enableSuite[GlutenParquetV1FilterSuite] + // Rewrite. + .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .exclude("SPARK-38825: in and notIn filters") + // TODO: fix in Spark-4.0 + .exclude("SPARK-47120: subquery literal filter pushdown") + enableSuite[GlutenParquetV2FilterSuite] + // Rewrite. + .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .exclude("SPARK-38825: in and notIn filters") + // TODO: fix in Spark-4.0 + .exclude("SPARK-47120: subquery literal filter pushdown") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetIOSuite] + // Velox doesn't write file metadata into parquet file. + .exclude("Write Spark version into Parquet metadata") + // Exception. + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. + .exclude("SPARK-35640: int as long should throw schema incompatible error") + // Velox parquet reader not allow offset zero. + .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + // TODO: fix in Spark-4.0 + .exclude("SPARK-49991: Respect 'mapreduce.output.basename' to generate file names") + .exclude("SPARK-6330 regression test") + .exclude("SPARK-7837 Do not close output writer twice when commitTask() fails") + .exclude("explode nested lists crossing a rowgroup boundary") + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + enableSuite[GlutenParquetProtobufCompatibilitySuite] + enableSuite[GlutenParquetV1QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .exclude("SPARK-39833: pushed filters with project without filter columns") + .exclude("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV2QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV1SchemaPruningSuite] + enableSuite[GlutenParquetV2SchemaPruningSuite] + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + // [PATH_NOT_FOUND] Path does not exist: + // file:/opt/spark331/sql/core/src/test/resources/test-data/timestamp-nanos.parquet + // May require for newer spark.test.home + .excludeByPrefix("SPARK-40819") + // TODO: fix in Spark-4.0 + .excludeByPrefix("SPARK-46056") + .exclude("CANNOT_MERGE_SCHEMAS: Failed merging schemas") + enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. + .exclude("Read Parquet file generated by parquet-thrift") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenFileFormatWriterSuite] + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .exclude("aggregate push down - different data types") + enableSuite[GlutenOrcV1AggregatePushDownSuite] + .exclude("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV2AggregatePushDownSuite] + .exclude("nested column: Max(top level column) not push down") + .exclude("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenParquetCodecSuite] + // codec not supported in native + .exclude("write and read - file source parquet - codec: lz4_raw") + .exclude("write and read - file source parquet - codec: lz4raw") + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenHeaderCSVReadSchemaSuite] + .exclude("change column type from int to long") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenOrcReadSchemaSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + enableSuite[GlutenParquetReadSchemaSuite] + // TODO: fix in Spark-4.0 + .exclude("read float and double together") + .exclude("change column type from float to double") + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + // TODO: fix in Spark-4.0 + .exclude("read float and double together") + .exclude("change column type from float to double") + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenV1WriteCommandSuite] + // Rewrite to match SortExecTransformer. + .excludeByPrefix("SPARK-41914:") + enableSuite[GlutenEnsureRequirementsSuite] + + enableSuite[GlutenBroadcastJoinSuite] + .exclude("Shouldn't change broadcast join buildSide if user clearly specified") + .exclude("Shouldn't bias towards build right if user didn't specify") + .exclude("SPARK-23192: broadcast hint should be retained after using the cached data") + .exclude("broadcast join where streamed side's output partitioning is HashPartitioning") + + enableSuite[GlutenExistenceJoinSuite] + enableSuite[GlutenInnerJoinSuiteForceShjOn] + enableSuite[GlutenInnerJoinSuiteForceShjOff] + enableSuite[GlutenOuterJoinSuiteForceShjOn] + enableSuite[GlutenOuterJoinSuiteForceShjOff] + enableSuite[FallbackStrategiesSuite] + enableSuite[GlutenBroadcastExchangeSuite] + enableSuite[GlutenLocalBroadcastExchangeSuite] + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCoalesceShufflePartitionsSuite] + enableSuite[GlutenExchangeSuite] + // ColumnarShuffleExchangeExec does not support doExecute() method + .exclude("shuffling UnsafeRows in exchange") + // This test will re-run in GlutenExchangeSuite with shuffle partitions > 1 + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .exclude("do not replace hash aggregate if child does not have sort order") + .exclude("do not replace hash aggregate if there is no group-by column") + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSortSuite] + enableSuite[GlutenSQLAggregateFunctionSuite] + // spill not supported yet. + enableSuite[GlutenSQLWindowFunctionSuite] + .exclude("test with low buffer spill threshold") + enableSuite[GlutenTakeOrderedAndProjectSuite] + // TODO: fix in Spark-4.0 + .exclude("SPARK-47104: Non-deterministic expressions in projection") + enableSuite[GlutenSessionExtensionSuite] + enableSuite[TestFileSourceScanExecTransformer] + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("disable bucketing when the output doesn't contain all bucketing columns") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + .exclude("write bucketed data") + .exclude("write bucketed data with sortBy") + .exclude("write bucketed data without partitionBy") + .exclude("write bucketed data without partitionBy with sortBy") + .exclude("write bucketed data with bucketing disabled") + enableSuite[GlutenCreateTableAsSelectSuite] + // TODO Gluten can not catch the spark exception in Driver side. + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + .disable( + "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenInsertSuite] + // the native write staing dir is differnt with vanilla Spark for coustom partition paths + .exclude("SPARK-35106: Throw exception when rename custom partition paths returns false") + .exclude("Stop task set if FileAlreadyExistsException was thrown") + // Rewrite: Additional support for file scan with default values has been added in Spark-3.4. + // It appends the default value in record if it is not present while scanning. + // Velox supports default values for new records but it does not backfill the + // existing records and provides null for the existing ones. + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + // TODO: fix in Spark-4.0 + .exclude("Throw exceptions on inserting out-of-range decimal value with ANSI casting policy") + .exclude("Throw exceptions on inserting out-of-range long value with ANSI casting policy") + .exclude("Throw exceptions on inserting out-of-range int value with ANSI casting policy") + enableSuite[GlutenPartitionedWriteSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + enableSuite[GlutenApproximatePercentileQuerySuite] + // requires resource files from Vanilla spark jar + .exclude("SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenCachedTableSuite] + .exclude("A cached table preserves the partitioning and ordering of its cached SparkPlan") + .exclude("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + // TODO: fix in Spark-4.0 + // enableSuite[GlutenFileSourceCharVarcharTestSuite] + // .exclude("length check for input string values: nested in array") + // .exclude("length check for input string values: nested in array") + // .exclude("length check for input string values: nested in map key") + // .exclude("length check for input string values: nested in map value") + // .exclude("length check for input string values: nested in both map key and value") + // .exclude("length check for input string values: nested in array of struct") + // .exclude("length check for input string values: nested in array of array") + // enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenColumnExpressionSuite] + // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. + // The final caught Spark exception's getCause().getMessage() contains 'errMsg' but does not + // equal 'errMsg' exactly. The following two tests will be skipped and overridden in Gluten. + .exclude("raise_error") + .exclude("assert_true") + enableSuite[GlutenComplexTypeSuite] + enableSuite[GlutenConfigBehaviorSuite] + // Will be fixed by cleaning up ColumnarShuffleExchangeExec. + .exclude("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + // Gluten columnar operator will have different number of jobs + .exclude("SPARK-40211: customize initialNumPartitions for take") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCTEHintSuite] + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenDataFrameAggregateSuite] + // Test for vanilla spark codegen, not apply for Gluten + .exclude("SPARK-43876: Enable fast hashmap for distinct queries") + .exclude( + "zero moments", // [velox does not return NaN] + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // incorrect result, distinct NaN case + "SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", + // Replaced with another test. + "SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it", + // Velox's collect_list / collect_set are by design declarative aggregate so plan check + // for ObjectHashAggregateExec will fail. + "SPARK-22223: ObjectHashAggregate should not introduce unnecessary shuffle", + "SPARK-31620: agg with subquery (whole-stage-codegen = true)", + "SPARK-31620: agg with subquery (whole-stage-codegen = false)" + ) + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + // blocked by Velox-5768 + .exclude("aggregate function - array for primitive type containing null") + .exclude("aggregate function - array for non-primitive type") + // Rewrite this test because Velox sorts rows by key for primitive data types, which disrupts the original row sequence. + .exclude("map_zip_with function - map of primitive types") + // TODO: fix in Spark-4.0 + .exclude("map_concat function") + .exclude("transform keys function - primitive data types") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite] + enableSuite[GlutenDataFrameNaFunctionsSuite] + .exclude( + // NaN case + "replace nan with float", + "replace nan with double" + ) + enableSuite[GlutenDataFramePivotSuite] + // substring issue + .exclude("pivot with column definition in groupby") + // array comparison not supported for values that contain nulls + .exclude( + "pivot with null and aggregate type not supported by PivotFirst returns correct result") + enableSuite[GlutenDataFrameRangeSuite] + .exclude("SPARK-20430 Initialize Range parameters in a driver side") + .excludeByPrefix("Cancelling stage in a query with Range") + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + enableSuite[GlutenDataFrameSetOperationsSuite] + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + enableSuite[GlutenDataFrameStatSuite] + // TODO: fix in Spark-4.0 + .exclude("Bloom filter") + enableSuite[GlutenDataFrameSuite] + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + "distributeBy and localSort", + // Mismatch when max NaN and infinite value + "NaN is greater than all other non-NaN numeric values", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // decimal failed ut. + "SPARK-22271: mean overflows and returns null for some decimal variables", + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + "SPARK-9083: sort with non-deterministic expressions" + ) + // test for sort node not present but gluten uses shuffle hash join + .exclude("SPARK-41048: Improve output partitioning and ordering with AQE cache") + // Rewrite this test since it checks the physical operator which is changed in Gluten + .exclude("SPARK-27439: Explain result should match collected result after view change") + enableSuite[GlutenDataFrameTimeWindowingSuite] + enableSuite[GlutenDataFrameTungstenSuite] + enableSuite[GlutenDataFrameWindowFunctionsSuite] + // does not support `spark.sql.legacy.statisticalAggregate=true` (null -> NAN) + .exclude("corr, covar_pop, stddev_pop functions in specific window") + .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") + // does not support spill + .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") + .exclude("SPARK-21258: complex object in combination with spilling") + // rewrite `WindowExec -> WindowExecTransformer` + .exclude( + "SPARK-38237: require all cluster keys for child required distribution for window query") + enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .exclude("unbounded preceding/following range between with aggregation") + .exclude("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + // Rewrite the following two tests in GlutenDatasetSuite. + .exclude("dropDuplicates: columns with same column name") + .exclude("groupBy.as") + // TODO: fix in Spark-4.0 + .exclude("SPARK-23627: provide isEmpty in DataSet") + enableSuite[GlutenDateFunctionsSuite] + // The below two are replaced by two modified versions. + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + // Unsupported datetime format: specifier X is not supported by velox. + .exclude("to_timestamp with microseconds precision") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") + // Legacy mode is not supported and velox getTimestamp function does not throw + // exception when format is "yyyy-dd-aa". + .exclude("function to_date") + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + // TODO: fix in Spark-4.0 + .exclude("join key with multiple references on the filtering plan") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFileBasedDataSourceSuite] + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // error msg from velox is different & reader options is not supported, rewrite + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .exclude("Return correct results when data columns overlap with partition columns") + .exclude("Return correct results when data columns overlap with partition " + + "columns (nested data)") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support partition pruning") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("SPARK-41017: filter pushdown with nondeterministic predicates") + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenGeneratorFunctionSuite] + .exclude("SPARK-45171: Handle evaluated nondeterministic expression") + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .exclude("Merge runtime bloom filters") + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenFileSourceSQLInsertTestSuite] + enableSuite[GlutenDSV2SQLInsertTestSuite] + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .exclude("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .exclude("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .exclude("SPARK-28156: self-join should not miss cached view") + .exclude("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .exclude("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Not useful and time consuming. + .exclude("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + // exception test, rewritten in gluten + .exclude("the escape character is not allowed to end with") + // ORC related + .exclude("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .exclude("SPARK-38173: Quoted column cannot be recognized correctly when quotedRegexColumnNames is true") + // TODO: fix in Spark-4.0 + .exclude("SPARK-47939: Explain should work with parameterized queries") + // enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenStatisticsCollectionSuite] + // The output byte size of Velox is different + .exclude("SPARK-33687: analyze all tables in a specific database") + .exclude("column stats collection for null columns") + .exclude("analyze column command - result verification") + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + // TODO: fix in Spark-4.0 + .excludeByPrefix("SPARK-51738") + .excludeByPrefix("SPARK-43402") + .exclude("non-aggregated correlated scalar subquery") + .exclude("SPARK-18504 extra GROUP BY column in correlated scalar subquery is not permitted") + .exclude("SPARK-43402: FileSourceScanExec supports push down data filter with scalar subquery") + .exclude("SPARK-51738: IN subquery with struct type") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + // Rewrite with NaN test cases excluded. + .exclude("cases when literal is max") + enableSuite[GlutenXPathFunctionsSuite] + enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHiveSQLQuerySuite] + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCollapseProjectExecTransformerSuite] + enableSuite[GlutenSparkSessionExtensionSuite] + enableSuite[GlutenGroupBasedDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedDeleteFromTableSuite] + enableSuite[GlutenDataFrameToSchemaSuite] + enableSuite[GlutenDatasetUnpivotSuite] + enableSuite[GlutenLateralColumnAliasSuite] + enableSuite[GlutenParametersSuite] + enableSuite[GlutenResolveDefaultColumnsSuite] + enableSuite[GlutenSubqueryHintPropagationSuite] + enableSuite[GlutenUrlFunctionsSuite] + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") + enableSuite[GlutenBitmapExpressionsQuerySuite] + enableSuite[GlutenEmptyInSuite] + enableSuite[GlutenRuntimeNullChecksV2Writes] + enableSuite[GlutenTableOptionsConstantFoldingSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableSuite] + // TODO: fix in Spark-4.0 + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite] + // TODO: fix in Spark-4.0 + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite] + // FIXME: complex type result mismatch + .exclude("update nested struct fields") + .exclude("update char/varchar columns") + enableSuite[GlutenDeltaBasedUpdateTableSuite] + enableSuite[GlutenGroupBasedMergeIntoTableSuite] + // TODO: fix in Spark-4.0 + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenFileSourceCustomMetadataStructSuite] + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] + enableSuite[GlutenTableLocationSuite] + enableSuite[GlutenRemoveRedundantWindowGroupLimitsSuite] + enableSuite[GlutenSQLCollectLimitExecSuite] + enableSuite[GlutenBatchEvalPythonExecSuite] + // Replaced with other tests that check for native operations + .exclude("Python UDF: push down deterministic FilterExec predicates") + .exclude("Nested Python UDF: push down deterministic FilterExec predicates") + .exclude("Python UDF: no push down on non-deterministic") + .exclude("Python UDF: push down on deterministic predicates after the first non-deterministic") + enableSuite[GlutenExtractPythonUDFsSuite] + // Replaced with test that check for native operations + .exclude("Python UDF should not break column pruning/filter pushdown -- Parquet V1") + .exclude("Chained Scalar Pandas UDFs should be combined to a single physical node") + .exclude("Mixed Batched Python UDFs and Pandas UDF should be separate physical node") + .exclude("Independent Batched Python UDFs and Scalar Pandas UDFs should be combined separately") + .exclude("Dependent Batched Python UDFs and Scalar Pandas UDFs should not be combined") + .exclude("Python UDF should not break column pruning/filter pushdown -- Parquet V2") + enableSuite[GlutenStreamingQuerySuite] + // requires test resources that don't exist in Gluten repo + .exclude("detect escaped path and report the migration guide") + .exclude("ignore the escaped path check when the flag is off") + .excludeByPrefix("SPARK-51187") + // TODO: fix in Spark-4.0 + .excludeByPrefix("SPARK-49905") + .excludeByPrefix("SPARK-41199") + .excludeByPrefix("SPARK-41198") + enableSuite[GlutenQueryExecutionSuite] + // Rewritten to set root logger level to INFO so that logs can be parsed + .exclude("Logging plan changes for execution") + // Rewrite for transformed plan + .exclude("dumping query execution info to a file - explainMode=formatted") + // TODO: fix in Spark-4.0 + .exclude("SPARK-47289: extended explain info") + + override def getSQLQueryTestSettings: SQLQueryTestSettings = VeloxSQLQueryTestSettings +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala new file mode 100644 index 000000000000..70579c886248 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala @@ -0,0 +1,26 @@ +/* + * 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.spark + +import org.apache.spark.sql.GlutenTestsBaseTrait + +class GlutenSortShuffleSuite extends SortShuffleSuite with GlutenTestsBaseTrait { + override def beforeAll(): Unit = { + super.beforeAll() + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala new file mode 100644 index 000000000000..86ef1238965f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenApproxCountDistinctForIntervalsQuerySuite + extends ApproxCountDistinctForIntervalsQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala new file mode 100644 index 000000000000..eb82baa78dac --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala @@ -0,0 +1,26 @@ +/* + * 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.spark.sql + +class GlutenApproximatePercentileQuerySuite + extends ApproximatePercentileQuerySuite + with GlutenSQLTestsTrait { + + override def testFile(fileName: String): String = { + Thread.currentThread().getContextClassLoader.getResource(fileName).toString + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala new file mode 100644 index 000000000000..e07821857a50 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenBitmapExpressionsQuerySuite + extends BitmapExpressionsQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala new file mode 100644 index 000000000000..3eb59d8feca9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala @@ -0,0 +1,154 @@ +/* + * 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.spark.sql + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal.SQLConf + +class GlutenBloomFilterAggregateQuerySuite + extends BloomFilterAggregateQuerySuite + with GlutenSQLTestsTrait + with AdaptiveSparkPlanHelper { + import testImplicits._ + + val veloxBloomFilterMaxNumBits = 4194304L + + testGluten("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") { + val table = "bloom_filter_test" + withSQLConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS.key -> "5000000") { + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT every(might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), + | col)) positive_membership_test + |FROM $table + """.stripMargin + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + checkAnswer(spark.sql(sqlString), Row(true)) + } + } + } + + testGluten("Test that might_contain on bloom_filter_agg with empty input") { + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), cast(123 as long))""".stripMargin), + Row(null) + ) + + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), null)""".stripMargin), + Row(null)) + } + + testGluten("Test bloom_filter_agg filter fallback") { + val table = "bloom_filter_test" + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT col positive_membership_test + |FROM $table + |WHERE might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), col) + """.stripMargin + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + withSQLConf( + GlutenConfig.COLUMNAR_PROJECT_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.size == 2, + df.queryExecution.executedPlan + ) + } + if (BackendsApiManager.getSettings.requireBloomFilterAggMightContainJointFallback()) { + withSQLConf( + GlutenConfig.COLUMNAR_FILTER_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.size == 2, + df.queryExecution.executedPlan + ) + } + } + } + } + + testGluten("Test bloom_filter_agg agg fallback") { + val table = "bloom_filter_test" + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT col positive_membership_test + |FROM $table + |WHERE might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), col) + """.stripMargin + + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + withSQLConf( + GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.isEmpty, + df.queryExecution.executedPlan + ) + } + } + } +} + +class GlutenBloomFilterAggregateQuerySuiteCGOff extends GlutenBloomFilterAggregateQuerySuite { + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.codegen.wholeStage", "false") + .set("spark.sql.codegen.factoryMode", "NO_CODEGEN") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala new file mode 100644 index 000000000000..8005bffc310d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenCTEHintSuite extends CTEHintSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala new file mode 100644 index 000000000000..3a05eda7119f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala @@ -0,0 +1,33 @@ +/* + * 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.spark.sql + +import org.apache.spark.SparkConf + +class GlutenCTEInlineSuiteAEOff extends CTEInlineSuiteAEOff with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + +} + +class GlutenCTEInlineSuiteAEOn extends CTEInlineSuiteAEOn with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala new file mode 100644 index 000000000000..0afabae6e5fd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -0,0 +1,158 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike +import org.apache.spark.sql.internal.SQLConf + +class GlutenCachedTableSuite + extends CachedTableSuite + with GlutenSQLTestsTrait + with AdaptiveSparkPlanHelper { + import testImplicits._ + // for temporarily disable the columnar table cache globally. + sys.props.put(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.shuffle.partitions", "5") + super.sparkConf.set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + } + + testGluten("InMemoryRelation statistics") { + sql("CACHE TABLE testData") + spark.table("testData").queryExecution.withCachedData.collect { + case cached: InMemoryRelation => + assert(cached.stats.sizeInBytes === 1132) + } + } + + def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + df.collect() + } + assert(collect(df.queryExecution.executedPlan) { + case e: ShuffleExchangeLike => e + }.size == expected) + } + + testGluten("A cached table preserves the partitioning and ordering of its cached SparkPlan") { + // Distribute the tables into non-matching number of partitions. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert( + stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 12) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. We'll only shuffle this side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // repartition's column ordering is different from group by column ordering. + // But they use the same set of columns. + withTempView("t1") { + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + spark.catalog.cacheTable("t1") + + val query = sql("SELECT value, key from t1 group by key, value") + verifyNumExchanges(query, 0) + checkAnswer(query, testData.distinct().select($"value", $"key")) + uncacheTable("t1") + } + + // repartition's column ordering is different from join condition's column ordering. + // We will still shuffle because hashcodes of a row depend on the column ordering. + // If we do not shuffle, we may actually partition two tables in totally two different way. + // See PartitioningSuite for more details. + withTempView("t1", "t2") { + val df1 = testData + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + val df2 = testData2.select($"a", $"b".cast("string")) + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = + sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a and t1.value = t2.b") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala new file mode 100644 index 000000000000..ce2f1b465e7f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -0,0 +1,123 @@ +/* + * 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.spark.sql + +import org.apache.spark.SparkException + +class GlutenFileSourceCharVarcharTestSuite + extends FileSourceCharVarcharTestSuite + with GlutenSQLTestsTrait { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + private val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + testGluten("length check for input string values: nested in struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT) USING $format") + sql("INSERT INTO t SELECT struct(null)") + checkAnswer(spark.table("t"), Row(Row(null))) + val e = intercept[RuntimeException] { + sql("INSERT INTO t SELECT struct('123456')") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map key") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), STRING>) USING $format") + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP) USING $format") + sql("INSERT INTO t VALUES (map('a', null))") + checkAnswer(spark.table("t"), Row(Map("a" -> null))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in both map key and value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), $typeName(5)>) USING $format") + val e1 = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e1.getMessage.contains(ERROR_MESSAGE)) + val e2 = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e2.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in struct of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT>) USING $format") + sql("INSERT INTO t SELECT struct(array(null))") + checkAnswer(spark.table("t"), Row(Row(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t SELECT struct(array('123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(struct(null)))") + checkAnswer(spark.table("t"), Row(Seq(Row(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(struct('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(array(null)))") + checkAnswer(spark.table("t"), Row(Seq(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(array('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } +} + +class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala new file mode 100644 index 000000000000..6ee95026fdfb --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -0,0 +1,103 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkRuntimeException +import org.apache.spark.sql.execution.ProjectExec +import org.apache.spark.sql.functions.{assert_true, expr, input_file_name, lit} + +class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTestsTrait { + import testImplicits._ + + // TODO: fix in Spark-4.0 + // ignore("raise_error") { + // val strDf = Seq(("hello")).toDF("a") + + // val e1 = intercept[SparkRuntimeException] { + // strDf.select(raise_error(lit(null.asInstanceOf[String]))).collect() + // } + // assert(e1.getCause.isInstanceOf[RuntimeException]) + + // val e2 = intercept[SparkRuntimeException] { + // strDf.select(raise_error($"a")).collect() + // } + // assert(e2.getCause.isInstanceOf[RuntimeException]) + // assert(e2.getCause.getMessage.contains("hello")) + // } + + // TODO: fix in Spark-4.0 + ignoreGluten("assert_true") { + // assert_true(condition, errMsgCol) + val booleanDf = Seq((true), (false)).toDF("cond") + checkAnswer( + booleanDf.filter("cond = true").select(assert_true($"cond")), + Row(null) :: Nil + ) + val e1 = intercept[SparkRuntimeException] { + booleanDf.select(assert_true($"cond", lit(null.asInstanceOf[String]))).collect() + } + assert(e1.getCause.isInstanceOf[RuntimeException]) + + val nullDf = Seq(("first row", None), ("second row", Some(true))).toDF("n", "cond") + checkAnswer( + nullDf.filter("cond = true").select(assert_true($"cond", $"cond")), + Row(null) :: Nil + ) + val e2 = intercept[SparkRuntimeException] { + nullDf.select(assert_true($"cond", $"n")).collect() + } + assert(e2.getCause.isInstanceOf[RuntimeException]) + assert(e2.getCause.getMessage.contains("first row")) + + // assert_true(condition) + val intDf = Seq((0, 1)).toDF("a", "b") + checkAnswer(intDf.select(assert_true($"a" < $"b")), Row(null) :: Nil) + val e3 = intercept[SparkRuntimeException] { + intDf.select(assert_true($"a" > $"b")).collect() + } + assert(e3.getCause.isInstanceOf[RuntimeException]) + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) + } + + testGluten( + "input_file_name, input_file_block_start and input_file_block_length " + + "should fall back if scan falls back") { + withSQLConf((GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false")) { + withTempPath { + dir => + val data = sparkContext.parallelize(0 to 10).toDF("id") + data.write.parquet(dir.getCanonicalPath) + + val q = + spark.read + .parquet(dir.getCanonicalPath) + .select( + input_file_name(), + expr("input_file_block_start()"), + expr("input_file_block_length()")) + val firstRow = q.head() + assert(firstRow.getString(0).contains(dir.toURI.getPath)) + assert(firstRow.getLong(1) == 0) + assert(firstRow.getLong(2) > 0) + val project = q.queryExecution.executedPlan.collect { case p: ProjectExec => p } + assert(project.size == 1) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala new file mode 100644 index 000000000000..835b7ecfd88b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.spark.sql + +class GlutenComplexTypesSuite extends ComplexTypesSuite with GlutenSQLTestsTrait { + + override def beforeAll(): Unit = { + super.beforeAll() + spark + .range(10) + .selectExpr( + "(id % 2 = 0) as bool", + "cast(id as BYTE) as i8", + "cast(id as SHORT) as i16", + "cast(id as FLOAT) as fp32", + "cast(id as DOUBLE) as fp64", + "cast(id as DECIMAL(4, 2)) as dec", + "cast(cast(id as BYTE) as BINARY) as vbin", + "binary(id) as vbin1", + "map_from_arrays(array(id),array(id+2)) as map", + "array(id, id+1, id+2) as list", + "struct(cast(id as LONG) as a, cast(id+1 as STRING) as b) as struct" + ) + .write + .saveAsTable("tab_types") + } + + override def afterAll(): Unit = { + try { + spark.sql("DROP TABLE IF EXISTS tab_types") + } finally { + super.afterAll() + } + } + + testGluten("types bool/byte/short/float/double/decimal/binary/map/array/struct") { + val df = spark + .table("tab_types") + .selectExpr( + "bool", + "i8", + "i16", + "fp32", + "fp64", + "dec", + "vbin", + "length(vbin)", + "vbin1", + "length(vbin1)", + "struct", + "struct.a", + "list", + "map" + ) + .sort("i8") + .limit(1) + + checkAnswer( + df, + Seq( + Row( + true, + 0.toByte, + 0.toShort, + 0.toFloat, + 0.toDouble, + BigDecimal(0), + Array.fill[Byte](1)(0.toByte), + 1.toInt, + Array.fill[Byte](8)(0.toByte), + 8.toInt, + Row(0.toLong, "1"), + 0.toLong, + Array(0, 1, 2), + Map(0 -> 2) + )) + ) + + checkNamedStruct(df.queryExecution.optimizedPlan, expectedCount = 0) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala new file mode 100644 index 000000000000..c1984a5e22dd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenConfigBehaviorSuite extends ConfigBehaviorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala new file mode 100644 index 000000000000..182464c0a5ee --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql + +/** End-to-end test suite for count_min_sketch. */ +class GlutenCountMinSketchAggQuerySuite + extends CountMinSketchAggQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala new file mode 100644 index 000000000000..0550fef442ff --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenCsvFunctionsSuite extends CsvFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala new file mode 100644 index 000000000000..2f3777caa174 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala @@ -0,0 +1,283 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +import java.lang.{Long => JLong} + +import scala.util.Random + +class GlutenDataFrameAggregateSuite extends DataFrameAggregateSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + // blackTestNameList is defined in ClickHouseNotSupport + + testGluten("count") { + // agg with no input col + assert(testData2.count() === testData2.rdd.map(_ => 1).count()) + + checkAnswer( + testData2.agg(count($"a"), sum_distinct($"a")), // non-partial + Row(6, 6.0)) + } + + testGluten("null count") { + checkAnswer(testData3.groupBy($"a").agg(count($"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer(testData3.groupBy($"a").agg(count($"a" + $"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer( + testData3 + .agg(count($"a"), count($"b"), count(lit(1)), count_distinct($"a"), count_distinct($"b")), + Row(2, 1, 2, 2, 1)) + + // [wishlist] does not support sum distinct +// checkAnswer( +// testData3.agg(count($"b"), count_distinct($"b"), sum_distinct($"b")), // non-partial +// Row(1, 1, 2) +// ) + } + + testGluten("groupBy") { + checkAnswer(testData2.groupBy("a").agg(sum($"b")), Seq(Row(1, 3), Row(2, 3), Row(3, 3))) + checkAnswer(testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum($"totB")), Row(9)) + checkAnswer(testData2.groupBy("a").agg(count("*")), Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("*" -> "count")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("b" -> "sum")), + Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil) + + val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) + .toDF("key", "value1", "value2", "rest") + + checkAnswer(df1.groupBy("key").min(), df1.groupBy("key").min("value1", "value2").collect()) + checkAnswer(df1.groupBy("key").min("value2"), Seq(Row("a", 0), Row("b", 4))) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3))) +// ) +// +// val decimalDataWithNulls = spark.sparkContext.parallelize( +// DecimalData(1, 1) :: +// DecimalData(1, null) :: +// DecimalData(2, 1) :: +// DecimalData(2, null) :: +// DecimalData(3, 1) :: +// DecimalData(3, 2) :: +// DecimalData(null, 2) :: Nil).toDF() +// checkAnswer( +// decimalDataWithNulls.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3)), +// Row(null, new java.math.BigDecimal(2))) +// ) + } + + testGluten("average") { + + checkAnswer(testData2.agg(avg($"a"), mean($"a")), Row(2.0, 2.0)) + + checkAnswer( + testData2.agg(avg($"a"), sum_distinct($"a")), // non-partial and test deprecated version + Row(2.0, 6.0) :: Nil) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.agg(avg($"a")), +// Row(new java.math.BigDecimal(2))) +// +// checkAnswer( +// decimalData.agg(avg($"a"), sum_distinct($"a")), // non-partial +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) +// +// checkAnswer( +// decimalData.agg(avg($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2))) +// // non-partial +// checkAnswer( +// decimalData.agg( +// avg($"a" cast DecimalType(10, 2)), sum_distinct($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) + } + + ignoreGluten("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { + withTempView("view") { + Seq( + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("abellina", 1.0f), + ("abellina", 2.0f)).toDF("uid", "score").createOrReplaceTempView("view") + + val df = spark.sql("select uid, count(distinct score) from view group by 1 order by 1 asc") + checkAnswer(df, Row("abellina", 2) :: Row("mithunr", 1) :: Nil) + } + } + + testGluten("variance") { + checkAnswer( + testData2.agg(var_samp($"a"), var_pop($"a"), variance($"a")), + Row(0.8, 2.0 / 3.0, 0.8)) + checkAnswer(testData2.agg(var_samp("a"), var_pop("a"), variance("a")), Row(0.8, 2.0 / 3.0, 0.8)) + } + + testGluten("aggregation with filter") { + Seq( + ("mithunr", 12.3f, 5.0f, true, 9.4f), + ("mithunr", 15.5f, 4.0f, false, 19.9f), + ("mithunr", 19.8f, 3.0f, false, 35.6f), + ("abellina", 20.1f, 2.0f, true, 98.0f), + ("abellina", 20.1f, 1.0f, true, 0.5f), + ("abellina", 23.6f, 2.0f, true, 3.9f) + ) + .toDF("uid", "time", "score", "pass", "rate") + .createOrReplaceTempView("view") + var df = spark.sql("select count(score) filter (where pass) from view group by time") + checkAnswer(df, Row(1) :: Row(0) :: Row(0) :: Row(2) :: Row(1) :: Nil) + + df = spark.sql("select count(score) filter (where pass) from view") + checkAnswer(df, Row(4) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view group by time") + checkAnswer(df, Row(0) :: Row(0) :: Row(1) :: Row(1) :: Row(0) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view") + checkAnswer(df, Row(2) :: Nil) + } + + testGluten("extend with cast expression") { + checkAnswer( + decimalData.agg( + sum($"a".cast("double")), + avg($"b".cast("double")), + count_distinct($"a"), + count_distinct($"b")), + Row(12.0, 1.5, 3, 2)) + } + + // This test is applicable to velox backend. For CH backend, the replacement is disabled. + testGluten("use gluten hash agg to replace vanilla spark sort agg") { + + withSQLConf((GlutenConfig.COLUMNAR_FORCE_HASHAGG_ENABLED.key, "false")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + // SortAggregateExec is expected to be used for string type input. + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + assert(find(df.queryExecution.executedPlan)(_.isInstanceOf[SortAggregateExec]).isDefined) + } + + withSQLConf((GlutenConfig.COLUMNAR_FORCE_HASHAGG_ENABLED.key, "true")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + // Sort agg is expected to be replaced by gluten's hash agg. + assert( + find(df.queryExecution.executedPlan)( + _.isInstanceOf[HashAggregateExecBaseTransformer]).isDefined) + } + } + + testGluten("mixed supported and unsupported aggregate functions") { + withUserDefinedFunction(("udaf_sum", true)) { + spark.udf.register( + "udaf_sum", + udaf(new Aggregator[JLong, JLong, JLong] { + override def zero: JLong = 0 + override def reduce(b: JLong, a: JLong): JLong = a + b + override def merge(b1: JLong, b2: JLong): JLong = b1 + b2 + override def finish(reduction: JLong): JLong = reduction + override def bufferEncoder: Encoder[JLong] = Encoders.LONG + override def outputEncoder: Encoder[JLong] = Encoders.LONG + }) + ) + + val df = spark.sql("SELECT a, udaf_sum(b), max(b) FROM testData2 group by a") + checkAnswer(df, Row(1, 3, 2) :: Row(2, 3, 2) :: Row(3, 3, 2) :: Nil) + } + } + + // Ported from spark DataFrameAggregateSuite only with plan check changed. + private def assertNoExceptions(c: Column): Unit = { + for ( + (wholeStage, useObjectHashAgg) <- + Seq((true, true), (true, false), (false, true), (false, false)) + ) { + withSQLConf( + (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString), + (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString)) { + + val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y") + + // test case for HashAggregate + val hashAggDF = df.groupBy("x").agg(c, sum("y")) + hashAggDF.collect() + val hashAggPlan = hashAggDF.queryExecution.executedPlan + if (wholeStage) { + assert(find(hashAggPlan) { + case WholeStageCodegenExec(_: HashAggregateExec) => true + // If offloaded, spark whole stage codegen takes no effect and a gluten hash agg is + // expected to be used. + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } else { + assert( + stripAQEPlan(hashAggPlan).isInstanceOf[HashAggregateExec] || + stripAQEPlan(hashAggPlan).find { + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } + + // test case for ObjectHashAggregate and SortAggregate + val objHashAggOrSortAggDF = df.groupBy("x").agg(c, collect_list("y")) + objHashAggOrSortAggDF.collect() + assert(stripAQEPlan(objHashAggOrSortAggDF.queryExecution.executedPlan).find { + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } + } + } + + testGluten( + "SPARK-19471: AggregationIterator does not initialize the generated" + + " result projection before using it") { + Seq( + monotonically_increasing_id(), + spark_partition_id(), + rand(Random.nextLong()), + randn(Random.nextLong()) + ).foreach(assertNoExceptions) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala new file mode 100644 index 000000000000..9367fab17f2b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameAsOfJoinSuite extends DataFrameAsOfJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala new file mode 100644 index 000000000000..7464968cba51 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameComplexTypeSuite extends DataFrameComplexTypeSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala new file mode 100644 index 000000000000..2b0b40790a76 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala @@ -0,0 +1,52 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.functions._ + +class GlutenDataFrameFunctionsSuite extends DataFrameFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("map_zip_with function - map of primitive types") { + val df = Seq( + (Map(8 -> 6L, 3 -> 5L, 6 -> 2L), Map[Integer, Integer]((6, 4), (8, 2), (3, 2))), + (Map(10 -> 6L, 8 -> 3L), Map[Integer, Integer]((8, 4), (4, null))), + (Map.empty[Int, Long], Map[Integer, Integer]((5, 1))), + (Map(5 -> 1L), null) + ).toDF("m1", "m2") + + GlutenQueryTestUtil.sameRows( + df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> k == v1 + v2)").collect.toSeq, + Seq( + Row(Map(8 -> true, 3 -> false, 6 -> true)), + Row(Map(10 -> null, 8 -> false, 4 -> null)), + Row(Map(5 -> null)), + Row(null)), + false + ) + + GlutenQueryTestUtil.sameRows( + df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)).collect.toSeq, + Seq( + Row(Map(8 -> true, 3 -> false, 6 -> true)), + Row(Map(10 -> null, 8 -> false, 4 -> null)), + Row(Map(5 -> null)), + Row(null)), + false + ) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala new file mode 100644 index 000000000000..663a6111b0d0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameHintSuite extends DataFrameHintSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala new file mode 100644 index 000000000000..2a6e367bc08a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameImplicitsSuite extends DataFrameImplicitsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala new file mode 100644 index 000000000000..6581d7f2d88d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala @@ -0,0 +1,24 @@ +/* + * 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.spark.sql + +class GlutenDataFrameJoinSuite extends DataFrameJoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Supports multi-part names for broadcast hint resolution" + ) +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala new file mode 100644 index 000000000000..424087c8de89 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameNaFunctionsSuite extends DataFrameNaFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala new file mode 100644 index 000000000000..e1b91d719976 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala @@ -0,0 +1,42 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.functions._ + +class GlutenDataFramePivotSuite extends DataFramePivotSuite with GlutenSQLTestsTrait { + + // This test is ported from vanilla spark with pos value (1-based) changed from 0 to 1 for + // substring. In vanilla spark, pos=0 has same effectiveness as pos=1. But in velox, pos=0 + // will return an empty string as substring result. + testGluten("pivot with column definition in groupby - using pos=1") { + val df = courseSales + .groupBy(substring(col("course"), 1, 1).as("foo")) + .pivot("year", Seq(2012, 2013)) + .sum("earnings") + .queryExecution + .executedPlan + + checkAnswer( + courseSales + .groupBy(substring(col("course"), 1, 1).as("foo")) + .pivot("year", Seq(2012, 2013)) + .sum("earnings"), + Row("d", 15000.0, 48000.0) :: Row("J", 20000.0, 30000.0) :: Nil + ) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala new file mode 100644 index 000000000000..e8a424de5be1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala new file mode 100644 index 000000000000..61cc4bc4c080 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameSelfJoinSuite extends DataFrameSelfJoinSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala new file mode 100644 index 000000000000..d76d8b21cdcf --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenDataFrameSessionWindowingSuite + extends DataFrameSessionWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala new file mode 100644 index 000000000000..fe7958b67773 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala @@ -0,0 +1,29 @@ +/* + * 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.spark.sql + +import org.apache.spark.SparkConf + +class GlutenDataFrameSetOperationsSuite + extends DataFrameSetOperationsSuite + with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.aggregation.union", "false") + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala new file mode 100644 index 000000000000..bab8e9b83cb2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameStatSuite extends DataFrameStatSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala new file mode 100644 index 000000000000..069f308fdca0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala @@ -0,0 +1,434 @@ +/* + * 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.spark.sql + +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.SparkException +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData2 +import org.apache.spark.sql.types.StringType + +import java.io.ByteArrayOutputStream +import java.nio.charset.StandardCharsets + +import scala.util.Random + +class GlutenDataFrameSuite extends DataFrameSuite with GlutenSQLTestsTrait { + + testGluten("repartitionByRange") { + val partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + import testImplicits._ + val data1d = Random.shuffle(0.to(partitionNum - 1)) + val data2d = data1d.map(i => (i, data1d.size - i)) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".asc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".desc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, data1d.size - 1 - i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(42)) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(0, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(null), $"val".asc, rand()) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + // .repartitionByRange() assumes .asc by default if no explicit sort order is specified + checkAnswer( + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b") + .select(spark_partition_id().as("id"), $"a", $"b"), + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b".asc) + .select(spark_partition_id().as("id"), $"a", $"b") + ) + + // at least one partition-by expression must be specified + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size) + } + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size, Seq.empty: _*) + } + } + } + + testGluten("distributeBy and localSort") { + import testImplicits._ + val data = spark.sparkContext.parallelize((1 to 100).map(i => TestData2(i % 10, i))).toDF() + + /** partitionNum = 1 */ + var partitionNum = 1 + val original = testData.repartition(partitionNum) + assert(original.rdd.partitions.length == partitionNum) + + // Distribute into one partition and order by. This partition should contain all the values. + val df6 = data.repartition(partitionNum, $"a").sortWithinPartitions("b") + // Walk each partition and verify that it is sorted ascending and not globally sorted. + df6.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (!allSequential) { + throw new SparkException("Partition should contain all sequential values") + } + } + + /** partitionNum = 5 */ + partitionNum = 5 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df = original.repartition(partitionNum, $"key") + assert(df.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df.select()) + + // Distribute and order by. + val df4 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".desc) + // Walk each partition and verify that it is sorted descending and does not contain all + // the values. + df4.rdd.foreachPartition { + p => + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be globally ordered") + } + } + } + + /** partitionNum = 10 */ + partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df2 = original.repartition(partitionNum, $"key") + assert(df2.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df2.select()) + } + + // Group by the column we are distributed by. This should generate a plan with no exchange + // between the aggregates + val df3 = testData.repartition($"key").groupBy("key").count() + verifyNonExchangingAgg(df3) + verifyNonExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key", "value") + .count()) + + // Grouping by just the first distributeBy expr, need to exchange. + verifyExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key") + .count()) + + /** partitionNum = 2 */ + partitionNum = 2 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + // Distribute and order by with multiple order bys + val df5 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".asc, $"a".asc) + // Walk each partition and verify that it is sorted ascending + df5.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be all sequential") + } + } + } + + testGluten("reuse exchange") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { + val df = spark.range(100).toDF() + val join = df.join(df, "id") + val plan = join.queryExecution.executedPlan + checkAnswer(join, df) + assert(collect(join.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size === 1) + assert(collect(join.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size === 1) + val broadcasted = broadcast(join) + val join2 = join.join(broadcasted, "id").join(broadcasted, "id") + checkAnswer(join2, df) + assert(collect(join2.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size == 1) + assert(collect(join2.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size == 4) + } + } + + // TODO: fix in spark-4.0 + /** Failed to check WholeStageCodegenExec, so we rewrite the UT. */ + // testGluten("SPARK-22520: support code generation for large CaseWhen") { + // import org.apache.spark.sql.catalyst.dsl.expressions.StringToAttributeConversionHelper + // val N = 30 + // var expr1 = when(equalizer($"id", lit(0)), 0) + // var expr2 = when(equalizer($"id", lit(0)), 10) + // (1 to N).foreach { + // i => + // expr1 = expr1.when(equalizer($"id", lit(i)), -i) + // expr2 = expr2.when(equalizer($"id", lit(i + 10)), i) + // } + // val df = spark.range(1).select(expr1, expr2.otherwise(0)) + // checkAnswer(df, Row(0, 10) :: Nil) + // // We check WholeStageTransformer instead of WholeStageCodegenExec + // assert(df.queryExecution.executedPlan.find(_.isInstanceOf[WholeStageTransformer]).isDefined) + // } + import testImplicits._ + + private lazy val person2: DataFrame = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + + testGluten("describe") { + val describeResult = Seq( + Row("count", "4", "4", "4"), + Row("mean", null, "33.0", "178.0"), + Row("stddev", null, "19.148542155126762", "11.547005383792516"), + Row("min", "Alice", "16", "164"), + Row("max", "David", "60", "192") + ) + + val emptyDescribeResult = Seq( + Row("count", "0", "0", "0"), + Row("mean", null, null, null), + Row("stddev", null, null, null), + Row("min", null, null, null), + Row("max", null, null, null)) + + val aggResult = Seq( + Row("4", "33.0", "19.148542155126762", "16", "60") + ) + + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) + + Seq("true", "false").foreach { + ansiEnabled => + withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled) { + val describeAllCols = person2.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "name", "age", "height")) + checkAnswer(describeAllCols, describeResult) + // All aggregate value should have been cast to string + describeAllCols.collect().foreach { + row => + row.toSeq.foreach { + value => + if (value != null) { + assert( + value.isInstanceOf[String], + "expected string but found " + value.getClass) + } + } + } + + val describeOneCol = person2.describe("age") + assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) + val aggOneCol = person2.agg( + count("age").cast(StringType), + avg("age").cast(StringType), + stddev_samp("age").cast(StringType), + min("age").cast(StringType), + max("age").cast(StringType)) + checkAnswer(aggOneCol, aggResult) + + val describeNoCol = person2.select().describe() + assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _, _) => Row(s) }) + + val emptyDescription = person2.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "name", "age", "height")) + checkAnswer(emptyDescription, emptyDescribeResult) + } + } + } + + testGluten("Allow leading/trailing whitespace in string before casting") { + withSQLConf("spark.gluten.velox.castFromVarcharAddTrimNode" -> "true") { + def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { + checkAnswer(df, expectedResult) + assert( + find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) + } + + // scalastyle:off nonascii + Seq( + " 123", + "123 ", + " 123 ", + "\u2000123\n\n\n", + "123\r\r\r", + "123\f\f\f", + "123\u000C", + "123\u0000") + .toDF("col1") + .createOrReplaceTempView("t1") + // scalastyle:on nonascii + val expectedIntResult = Row(123) :: Row(123) :: + Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil + var df = spark.sql("select cast(col1 as int) from t1") + checkResult(df, expectedIntResult) + df = spark.sql("select cast(col1 as long) from t1") + checkResult(df, expectedIntResult) + + Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") + .toDF("col1") + .createOrReplaceTempView("t1") + val expectedFloatResult = Row(123.5) :: Row(123.5) :: + Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil + df = spark.sql("select cast(col1 as float) from t1") + checkResult(df, expectedFloatResult) + df = spark.sql("select cast(col1 as double) from t1") + checkResult(df, expectedFloatResult) + + // scalastyle:off nonascii + val rawData = + Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") + // scalastyle:on nonascii + rawData.toDF("col1").createOrReplaceTempView("t1") + val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq + df = spark.sql("select cast(col1 as binary) from t1") + checkResult(df, expectedBinaryResult) + } + } + + testGluten("SPARK-27439: Explain result should match collected result after view change") { + withTempView("test", "test2", "tmp") { + spark.range(10).createOrReplaceTempView("test") + spark.range(5).createOrReplaceTempView("test2") + spark.sql("select * from test").createOrReplaceTempView("tmp") + val df = spark.sql("select * from tmp") + spark.sql("select * from test2").createOrReplaceTempView("tmp") + + val captured = new ByteArrayOutputStream() + Console.withOut(captured) { + df.explain(extended = true) + } + checkAnswer(df, spark.range(10).toDF) + val output = captured.toString + assert(output.contains("""== Parsed Logical Plan == + |'Project [*] + |+- 'UnresolvedRelation [tmp]""".stripMargin)) + assert(output.contains("""== Physical Plan == + |*(1) ColumnarToRow + |+- ColumnarRange 0, 10, 1, 2, 10""".stripMargin)) + } + } + + // TODO: fix in spark-4.0 + // private def withExpr(newExpr: Expression): Column = new Column(newExpr) + + // def equalizer(expr: Expression, other: Any): Column = withExpr { + // val right = lit(other).expr + // if (expr == right) { + // logWarning( + // s"Constructing trivially true equals predicate, '$expr = $right'. " + + // "Perhaps you need to use aliases.") + // } + // EqualTo(expr, right) + // } + + private def verifyNonExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case agg: HashAggregateExec => + atFirstAgg = !atFirstAgg + case _ => + if (atFirstAgg) { + fail("Should not have operators between the two aggregations") + } + } + } + + private def verifyExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case _: HashAggregateExec => + if (atFirstAgg) { + fail("Should not have back to back Aggregates") + } + atFirstAgg = true + case _: ShuffleExchangeExec => atFirstAgg = false + case _ => + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala new file mode 100644 index 000000000000..f2833a357cd2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenDataFrameTimeWindowingSuite + extends DataFrameTimeWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala new file mode 100644 index 000000000000..d578b92c4c8a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameToSchemaSuite extends DataFrameToSchemaSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala new file mode 100644 index 000000000000..0e555c8eac6f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala @@ -0,0 +1,34 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.types._ + +class GlutenDataFrameTungstenSuite extends DataFrameTungstenSuite with GlutenSQLTestsTrait { + + testGluten("Map type with struct type as key") { + val kv = Map(Row(1, 2L) -> Seq("v")) + val data = sparkContext.parallelize(Seq(Row(1, kv))) + val schema = new StructType() + .add("a", IntegerType) + .add( + "b", + MapType(new StructType().add("k1", IntegerType).add("k2", LongType), ArrayType(StringType))) + val df = spark.createDataFrame(data, schema) + assert(df.select("b").first() === Row(kv)) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala new file mode 100644 index 000000000000..3ba990d2eea6 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenDataFrameWindowFramesSuite + extends DataFrameWindowFramesSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala new file mode 100644 index 000000000000..978685db87a0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala @@ -0,0 +1,209 @@ +/* + * 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.spark.sql + +import org.apache.gluten.execution.WindowExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.ENSURE_REQUIREMENTS +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +class GlutenDataFrameWindowFunctionsSuite + extends DataFrameWindowFunctionsSuite + with GlutenSQLTestsTrait { + + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + // avoid single partition + .set("spark.sql.shuffle.partitions", "2") + } + + testGluten("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544) + ) + ) + } + } + + testGluten("corr, covar_pop, stddev_pop functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + covar_pop("value1", "value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0) + ) + ) + } + } + + testGluten( + "SPARK-38237: require all cluster keys for child required distribution for window query") { + def partitionExpressionsColumns(expressions: Seq[Expression]): Seq[String] = { + expressions.flatMap { case ref: AttributeReference => Some(ref.name) } + } + + def isShuffleExecByRequirement( + plan: ColumnarShuffleExchangeExec, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ColumnarShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS, _, _) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + + val df = Seq(("a", 1, 1), ("a", 2, 2), ("b", 1, 3), ("b", 1, 4)).toDF("key1", "key2", "value") + val windowSpec = Window.partitionBy("key1", "key2").orderBy("value") + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION.key -> "true") { + + val windowed = df + // repartition by subset of window partitionBy keys which satisfies ClusteredDistribution + .repartition($"key1") + .select(lead($"key1", 1).over(windowSpec), lead($"value", 1).over(windowSpec)) + + checkAnswer(windowed, Seq(Row("b", 4), Row(null, null), Row(null, null), Row(null, null))) + + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExecTransformer => + w.child.exists { + case s: ColumnarShuffleExchangeExec => + isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false + } + + assert(shuffleByRequirement, "Can't find desired shuffle node from the query plan") + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala new file mode 100644 index 000000000000..ddae3139d06b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDataFrameWriterV2Suite extends DataFrameWriterV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala new file mode 100644 index 000000000000..8a9a6b5756e9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDatasetAggregatorSuite extends DatasetAggregatorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala new file mode 100644 index 000000000000..848560192722 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDatasetCacheSuite extends DatasetCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala new file mode 100644 index 000000000000..a9d1bd29cead --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDatasetOptimizationSuite extends DatasetOptimizationSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala new file mode 100644 index 000000000000..c7463dcef75f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDatasetPrimitiveSuite extends DatasetPrimitiveSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala new file mode 100644 index 000000000000..6749227ed79d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenDatasetSerializerRegistratorSuite + extends DatasetSerializerRegistratorSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala new file mode 100644 index 000000000000..a8e73cee5a81 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala @@ -0,0 +1,62 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec + +class GlutenDatasetSuite extends DatasetSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("dropDuplicates: columns with same column name") { + val ds1 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + val ds2 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + // The dataset joined has two columns of the same name "_2". + val joined = ds1.join(ds2, "_1").select(ds1("_2").as[Int], ds2("_2").as[Int]) + // Using the checkDatasetUnorderly method to sort the result in Gluten. + checkDatasetUnorderly(joined.dropDuplicates(), (1, 2), (1, 1), (2, 1), (2, 2)) + } + + testGluten("groupBy.as") { + val df1 = Seq(DoubleData(1, "one"), DoubleData(2, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + val df2 = Seq(DoubleData(5, "one"), DoubleData(1, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + + val df3 = df1 + .groupBy("id") + .as[Int, DoubleData] + .cogroup(df2.groupBy("id").as[Int, DoubleData]) { + case (key, data1, data2) => + if (key == 1) { + Iterator(DoubleData(key, (data1 ++ data2).foldLeft("")((cur, next) => cur + next.val1))) + } else Iterator.empty + } + checkDataset(df3, DoubleData(1, "onetwo")) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { + case h: ColumnarShuffleExchangeExec => h + } + // Assert the number of ColumnarShuffleExchangeExec + // instead of ShuffleExchangeExec in Gluten. + assert(exchanges.size == 2) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala new file mode 100644 index 000000000000..e3ba780530fd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDatasetUnpivotSuite extends DatasetUnpivotSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala new file mode 100644 index 000000000000..082f06641b5a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala @@ -0,0 +1,300 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +import java.sql.{Date, Timestamp} +import java.time.{LocalDateTime, ZoneId} +import java.util.concurrent.TimeUnit + +class GlutenDateFunctionsSuite extends DateFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + private def secs(millis: Long): Long = TimeUnit.MILLISECONDS.toSeconds(millis) + + testGluten("unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val ntzTs1 = LocalDateTime.parse("2015-07-24T10:00:00.3") + val ntzTs2 = LocalDateTime.parse("2015-07-25T02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, ntzTs1, s1, ss1), (date2, ts2, ntzTs2, s2, ss2)).toDF( + "d", + "ts", + "ntzTs", + "s", + "ss") + checkAnswer( + df.select(unix_timestamp(col("ts"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ss"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ntzTs"))), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.select(unix_timestamp(col("d"), fmt)), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("s"), fmt)), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ntzTs)"), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.selectExpr(s"unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.select(unix_timestamp(col("x"))), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr("unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), + Seq(Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // invalid format + val invalid = df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')") + checkAnswer(invalid, Seq(Row(null), Row(null), Row(null), Row(null))) + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + // Test unix_timestamp(timestamp, format) - format should be ignored + checkAnswer( + df.select(unix_timestamp(col("ts"), "yyyy-MM-dd")), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ts, 'invalid-format')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val now = sql("select unix_timestamp()").collect().head.getLong(0) + checkAnswer( + sql(s"select timestamp_seconds($now)"), + Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) + } + } + } + + testGluten("to_unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString + ) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer( + df.selectExpr("to_unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("to_unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.selectExpr("to_unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + // Test to_unix_timestamp(timestamp, format) - format should be ignored + checkAnswer( + df.selectExpr("to_unix_timestamp(ts, 'yyyy-MM-dd')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("to_unix_timestamp(ts, 'invalid-format')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val invalid = df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')") + checkAnswer(invalid, Seq(Row(null), Row(null), Row(null), Row(null))) + } + } + } + + testGluten("function to_date") { + val d1 = Date.valueOf("2015-07-22") + val d2 = Date.valueOf("2015-07-01") + val d3 = Date.valueOf("2014-12-31") + val t1 = Timestamp.valueOf("2015-07-22 10:00:00") + val t2 = Timestamp.valueOf("2014-12-31 23:59:59") + val t3 = Timestamp.valueOf("2014-12-31 23:59:59") + val s1 = "2015-07-22 10:00:00" + val s2 = "2014-12-31" + val s3 = "2014-31-12" + val df = Seq((d1, t1, s1), (d2, t2, s2), (d3, t3, s3)).toDF("d", "t", "s") + + checkAnswer( + df.select(to_date(col("t"))), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"))), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("s"))), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + checkAnswer( + df.selectExpr("to_date(t)"), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.selectExpr("to_date(d)"), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.selectExpr("to_date(s)"), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + // now with format + checkAnswer( + df.select(to_date(col("t"), "yyyy-MM-dd")), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"), "yyyy-MM-dd")), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + val confKey = SQLConf.LEGACY_TIME_PARSER_POLICY.key + withSQLConf(confKey -> "corrected") { + checkAnswer( + df.select(to_date(col("s"), "yyyy-MM-dd")), + Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) + } + // legacyParserPolicy is not respected by Gluten. + // withSQLConf(confKey -> "exception") { + // checkExceptionMessage(df.select(to_date(col("s"), "yyyy-MM-dd"))) + // } + + // now switch format + checkAnswer( + df.select(to_date(col("s"), "yyyy-dd-MM")), + Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31")))) + + // invalid format + checkAnswer(df.select(to_date(col("s"), "yyyy-hh-MM")), Seq(Row(null), Row(null), Row(null))) + // velox getTimestamp function does not throw exception when format is "yyyy-dd-aa". + // val e = + // intercept[SparkUpgradeException](df.select(to_date(col("s"), "yyyy-dd-aa")).collect()) + // assert(e.getCause.isInstanceOf[IllegalArgumentException]) + // assert( + // e.getMessage.contains("You may get a different result due to the upgrading to Spark")) + + // February + val x1 = "2016-02-29" + val x2 = "2017-02-29" + val df1 = Seq(x1, x2).toDF("x") + checkAnswer(df1.select(to_date(col("x"))), Row(Date.valueOf("2016-02-29")) :: Row(null) :: Nil) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala new file mode 100644 index 000000000000..b6428773f1d0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenDeprecatedAPISuite extends DeprecatedAPISuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala new file mode 100644 index 000000000000..df42c790995b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala @@ -0,0 +1,774 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer, FilterExecTransformerBase} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} +import org.apache.spark.sql.catalyst.plans.ExistenceJoin +import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, ReusedExchangeExec} +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +abstract class GlutenDynamicPartitionPruningSuiteBase + extends DynamicPartitionPruningSuiteBase + with GlutenSQLTestsTrait { + + import testImplicits._ + + override def beforeAll(): Unit = { + prepareWorkDir() + super.beforeAll() + spark.sparkContext.setLogLevel("WARN") + } + + override def testNameBlackList: Seq[String] = Seq( + // overwritten with different plan + "SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec", + "Make sure dynamic pruning works on uncorrelated queries", + "Subquery reuse across the whole plan" + ) + + // === Following cases override super class's cases === + + ignoreGluten("DPP should not be rewritten as an existential join") { + // ignored: BroadcastHashJoinExec is from Vanilla Spark + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "1.5", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(s""" + |SELECT * FROM product p WHERE p.store_id NOT IN + | (SELECT f.store_id FROM fact_sk f JOIN dim_store d ON + | f.store_id = d.store_id + | WHERE d.state_province = 'NL' + | ) + """.stripMargin) + + val found = df.queryExecution.executedPlan.find { + case _ @BroadcastHashJoinExec(_, _, _: ExistenceJoin, _, _, _, _, _) => true + case _ => false + } + + assert(found.isEmpty) + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("no partition pruning when the build side is a stream") { + withTable("fact") { + val input = MemoryStream[Int] + val stream = input.toDF.select($"value".as("one"), ($"value" * 3).as("code")) + spark + .range(100) + .select($"id", ($"id" + 1).as("one"), ($"id" + 2).as("two"), ($"id" + 3).as("three")) + .write + .partitionBy("one") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + val table = sql("SELECT * from fact f") + + // join a partitioned table with a stream + val joined = table.join(stream, Seq("one")).where("code > 40") + val query = joined.writeStream.format("memory").queryName("test").start() + input.addData(1, 10, 20, 40, 50) + try { + query.processAllAvailable() + } finally { + query.stop() + } + // search dynamic pruning predicates on the executed plan + val plan = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution.executedPlan + val ret = plan.find { + case s: FileSourceScanExecTransformer => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case s: FileSourceScanExec => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + } + assert(ret.isDefined == false) + } + } + + testGluten("Make sure dynamic pruning works on uncorrelated queries") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql(""" + |SELECT d.store_id, + | SUM(f.units_sold), + | (SELECT SUM(f.units_sold) + | FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + | WHERE d.country = 'US') AS total_prod + |FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + |WHERE d.country = 'US' + |GROUP BY 1 + """.stripMargin) + checkAnswer(df, Row(4, 50, 70) :: Row(5, 10, 70) :: Row(6, 10, 70) :: Nil) + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = + collectWithSubqueries(plan) { + case _: SubqueryBroadcastExec => 1 + case _: ColumnarSubqueryBroadcastExec => 1 + }.sum + + val countReusedSubqueryBroadcasts = + collectWithSubqueries(plan) { + case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 + case ReusedSubqueryExec(_: ColumnarSubqueryBroadcastExec) => 1 + }.sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) + } + } + + testGluten( + "SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql(""" WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = collect(df.queryExecution.executedPlan) { + case se: ReusedExchangeExec => se + } + assert( + reuseExchangeNodes.size == 1, + "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + + testGluten("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") { + withTable("duplicate_keys") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) + .toDF("store_id", "country") + .write + .format(tableFormat) + .saveAsTable("duplicate_keys") + + val df = sql(""" + |SELECT date_id, product_id FROM fact_sk f + |JOIN duplicate_keys s + |ON f.store_id = s.store_id WHERE s.country = 'US' AND date_id > 1050 + """.stripMargin) + + checkPartitionPruningPredicate(df, withSubquery = false, withBroadcast = true) + + val subqueryBroadcastExecs = collectWithSubqueries(df.queryExecution.executedPlan) { + case s: ColumnarSubqueryBroadcastExec => s + } + assert(subqueryBroadcastExecs.size === 1) + subqueryBroadcastExecs.foreach { + subqueryBroadcastExec => + assert(subqueryBroadcastExec.metrics("numOutputRows").value === 1) + } + + checkAnswer(df, Row(1060, 2) :: Row(1060, 2) :: Row(1060, 2) :: Nil) + } + } + } + + // === Following methods override super class's methods === + + override protected def collectDynamicPruningExpressions(plan: SparkPlan): Seq[Expression] = { + flatMap(plan) { + case s: FileSourceScanExecTransformer => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: FileSourceScanExec => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExecTransformer => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExec => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case _ => Nil + } + } + + override def checkPartitionPruningPredicate( + df: DataFrame, + withSubquery: Boolean, + withBroadcast: Boolean): Unit = { + df.collect() + + val plan = df.queryExecution.executedPlan + val dpExprs = collectDynamicPruningExpressions(plan) + val hasSubquery = dpExprs.exists { + case InSubqueryExec(_, _: SubqueryExec, _, _, _, _) => true + case _ => false + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" + assert( + hasSubquery == withSubquery, + s"$hasFilter trigger DPP with a subquery duplicate:\n${df.queryExecution}") + val hasBroadcast = if (withBroadcast) "Should" else "Shouldn't" + assert( + subqueryBroadcast.nonEmpty == withBroadcast, + s"$hasBroadcast trigger DPP with a reused broadcast exchange:\n${df.queryExecution}") + + subqueryBroadcast.foreach { + s => + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => // reuse check ok. + case b: BroadcastExchangeLike => + val hasReuse = plan.find { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case a: AdaptiveSparkPlanExec => + val broadcastQueryStage = collectFirst(a) { case b: BroadcastQueryStageExec => b } + val broadcastPlan = broadcastQueryStage.get.broadcast + val hasReuse = find(plan) { + case ReusedExchangeExec(_, e) => e eq broadcastPlan + case b: BroadcastExchangeLike => b eq broadcastPlan + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case _ => + fail(s"Invalid child node found in\n$s") + } + } + + val isMainQueryAdaptive = plan.isInstanceOf[AdaptiveSparkPlanExec] + subqueriesAll(plan).filterNot(subqueryBroadcast.contains).foreach { + s => + val subquery = s match { + case r: ReusedSubqueryExec => r.child + case o => o + } + assert( + subquery.find(_.isInstanceOf[AdaptiveSparkPlanExec]).isDefined == isMainQueryAdaptive) + } + } + + override def checkDistinctSubqueries(df: DataFrame, n: Int): Unit = { + df.collect() + + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.indices + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => + b.indices + } + assert(buf.distinct.size == n) + } + + override def checkUnpushedFilters(df: DataFrame): Boolean = { + find(df.queryExecution.executedPlan) { + case FilterExec(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case transformer: FilterExecTransformerBase => + splitConjunctivePredicates(transformer.cond).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case FilterTransformer(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + }.isDefined + } + + object FilterTransformer { + def unapply(plan: SparkPlan): Option[(Expression, SparkPlan)] = { + plan match { + case transformer: FilterExecTransformerBase => + Some((transformer.cond, transformer.input)) + case _ => None + } + } + } +} + +abstract class GlutenDynamicPartitionPruningV1Suite extends GlutenDynamicPartitionPruningSuiteBase { + + import testImplicits._ + + /** Check the static scan metrics with and without DPP */ + // TODO: fix in Spark-4.0 + ignoreGluten("static scan metrics", DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: FileSourceScanExecTransformer => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case s: BatchScanExecTransformer => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + def getDriverMetrics(plan: SparkPlan, key: String): Option[SQLMetric] = plan match { + case fs: FileSourceScanExec => fs.driverMetrics.get(key) + case fs: FileSourceScanExecTransformer => fs.driverMetrics.get(key) + case _ => None + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + + val allFilesNum: Long = getDriverMetrics(scan1, "numFiles").map(_.value).getOrElse(-1) + val allFilesSize: Long = getDriverMetrics(scan1, "filesSize").map(_.value).getOrElse(-1) + val allPartitions: Long = + getDriverMetrics(scan1, "numPartitions").map(_.value).getOrElse(-1) + assert(allPartitions === numPartitions) + val pruningTimeVal1: Long = + getDriverMetrics(scan1, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal1 === 0) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum: Long = getDriverMetrics(scan2, "numFiles").map(_.value).getOrElse(-1) + val partFilesSize: Long = getDriverMetrics(scan2, "filesSize").map(_.value).getOrElse(-1) + val partPartitions: Long = + getDriverMetrics(scan2, "numPartitions").map(_.value).getOrElse(-1) + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(partPartitions === 1) + val pruningTimeVal2: Long = + getDriverMetrics(scan2, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal2 === 0) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + val staticFilesNumVal: Long = + getDriverMetrics(scan3, "staticFilesNum").map(_.value).getOrElse(-1) + val staticFilesSizeVal: Long = + getDriverMetrics(scan3, "staticFilesSize").map(_.value).getOrElse(-1) + val numFilesVal: Long = getDriverMetrics(scan3, "numFiles").map(_.value).getOrElse(-1) + val filesSizeVal: Long = getDriverMetrics(scan3, "filesSize").map(_.value).getOrElse(-1) + val numPartitionsVal: Long = + getDriverMetrics(scan3, "numPartitions").map(_.value).getOrElse(-1) + val pruningTimeVal3: Long = + getDriverMetrics(scan3, "pruningTime").map(_.value).getOrElse(-1) + assert(staticFilesNumVal == allFilesNum) + assert(staticFilesSizeVal == allFilesSize) + assert(numFilesVal == partFilesNum) + assert(filesSizeVal == partFilesSize) + assert(numPartitionsVal === 1) + assert(pruningTimeVal3 > -1) + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOff + extends GlutenDynamicPartitionPruningV1Suite + with DisableAdaptiveExecutionSuite { + + import testImplicits._ + + // TODO: fix in Spark-4.0 + ignoreGluten( + "override static scan metrics", + DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + // "spark.gluten.enabled" -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExecTransformer => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExecTransformer => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + def getDriverMetrics(plan: SparkPlan, key: String): Option[SQLMetric] = plan match { + case fs: FileSourceScanExec => fs.driverMetrics.get(key) + case fs: FileSourceScanExecTransformer => fs.driverMetrics.get(key) + case _ => None + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + val allFilesNum: Long = getDriverMetrics(scan1, "numFiles").map(_.value).getOrElse(-1) + val allFilesSize: Long = getDriverMetrics(scan1, "filesSize").map(_.value).getOrElse(-1) + val allPartitions: Long = + getDriverMetrics(scan1, "numPartitions").map(_.value).getOrElse(-1) + assert(allPartitions === numPartitions) + val pruningTimeVal1: Long = + getDriverMetrics(scan1, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal1 === 0) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum: Long = getDriverMetrics(scan2, "numFiles").map(_.value).getOrElse(-1) + val partFilesSize: Long = getDriverMetrics(scan2, "filesSize").map(_.value).getOrElse(-1) + val partPartitions: Long = + getDriverMetrics(scan2, "numPartitions").map(_.value).getOrElse(-1) + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(partPartitions === 1) + val pruningTimeVal2: Long = + getDriverMetrics(scan2, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal2 === 0) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + val staticFilesNumVal: Long = + getDriverMetrics(scan3, "staticFilesNum").map(_.value).getOrElse(-1) + val staticFilesSizeVal: Long = + getDriverMetrics(scan3, "staticFilesSize").map(_.value).getOrElse(-1) + val numFilesVal: Long = getDriverMetrics(scan3, "numFiles").map(_.value).getOrElse(-1) + val filesSizeVal: Long = getDriverMetrics(scan3, "filesSize").map(_.value).getOrElse(-1) + val numPartitionsVal: Long = + getDriverMetrics(scan3, "numPartitions").map(_.value).getOrElse(-1) + val pruningTimeVal3: Long = + getDriverMetrics(scan3, "pruningTime").map(_.value).getOrElse(-1) + assert(staticFilesNumVal == allFilesNum) + assert(staticFilesSizeVal == allFilesSize) + assert(numFilesVal == partFilesNum) + assert(filesSizeVal == partFilesSize) + assert(numPartitionsVal === 1) + assert(pruningTimeVal3 > -1) + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten( + "Subquery reuse across the whole plan", + DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("df1", "df2") { + spark + .range(100) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df1") + + spark + .range(10) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df2") + + val df = sql(""" + |SELECT df1.id, df2.k + |FROM df1 JOIN df2 ON df1.k = df2.k + |WHERE df2.id < (SELECT max(id) FROM df2 WHERE id <= 2) + |""".stripMargin) + + checkPartitionPruningPredicate(df, true, false) + + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) + + val plan = df.queryExecution.executedPlan + + val subqueryIds = plan.collectWithSubqueries { case s: SubqueryExec => s.id } + val reusedSubqueryIds = plan.collectWithSubqueries { + case rs: ReusedSubqueryExec => rs.child.id + } + + // By default Gluten pushes more filters than vanilla Spark. + // + // See also org.apache.gluten.execution.FilterHandler#applyFilterPushdownToScan + // See also DynamicPartitionPruningSuite.scala:1362 + assert(subqueryIds.size == 3, "Whole plan subquery reusing not working correctly") + assert(reusedSubqueryIds.size == 2, "Whole plan subquery reusing not working correctly") + assert( + reusedSubqueryIds.forall(subqueryIds.contains(_)), + "ReusedSubqueryExec should reuse an existing subquery") + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOn + extends GlutenDynamicPartitionPruningV1Suite + with EnableAdaptiveExecutionSuite { + + testGluten("SPARK-39447: Avoid AssertionError in AdaptiveSparkPlanExec.doExecuteBroadcast") { + val df = sql(""" + |WITH empty_result AS ( + | SELECT * FROM fact_stats WHERE product_id < 0 + |) + |SELECT * + |FROM (SELECT /*+ SHUFFLE_MERGE(fact_sk) */ empty_result.store_id + | FROM fact_sk + | JOIN empty_result + | ON fact_sk.product_id = empty_result.product_id) t2 + | JOIN empty_result + | ON t2.store_id = empty_result.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + checkAnswer(df, Nil) + } + + testGluten( + "SPARK-37995: PlanAdaptiveDynamicPruningFilters should use prepareExecutedPlan " + + "rather than createSparkPlan to re-plan subquery") { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(""" + |SELECT f.date_id, f.store_id FROM fact_sk f + |JOIN dim_store s ON f.store_id = s.store_id AND s.country = 'NL' + |WHERE s.state_province != (SELECT max(state_province) FROM dim_stats) + """.stripMargin) + + checkPartitionPruningPredicate(df, true, false) + checkAnswer(df, Row(1000, 1) :: Row(1010, 2) :: Row(1020, 2) :: Nil) + } + } + + testGluten("Filter with DynamicPruningExpression") { + withTable("fact_stats_non_partition") { + spark + .table("fact_stats") + .write + .format(tableFormat) + .saveAsTable("fact_stats_non_partition") + + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { + val df = sql(""" + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM ( + | select * from fact_stats + | union all + | select * from fact_stats_non_partition + |) f + |JOIN dim_stats s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) + checkAnswer( + df, + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: Nil + ) + val filters = collect(df.queryExecution.executedPlan) { case f: FilterExec => f } + assert(filters.isEmpty) + val filterTransformerWithDPPs = collect(df.queryExecution.executedPlan) { + case f: FilterExecTransformerBase + if f.cond.exists(_.isInstanceOf[DynamicPruningExpression]) => + f + } + assert(filterTransformerWithDPPs.nonEmpty) + } + } + } +} + +abstract class GlutenDynamicPartitionPruningV2Suite extends GlutenDynamicPartitionPruningSuiteBase { + override protected def runAnalyzeColumnCommands: Boolean = false + + override protected def initState(): Unit = { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + spark.conf.set("spark.sql.defaultCatalog", "testcat") + } +} + +class GlutenDynamicPartitionPruningV2SuiteAEOff + extends GlutenDynamicPartitionPruningV2Suite + with DisableAdaptiveExecutionSuite + +class GlutenDynamicPartitionPruningV2SuiteAEOn + extends GlutenDynamicPartitionPruningV2Suite + with EnableAdaptiveExecutionSuite + +// Test DPP with file scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV1SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Test DPP with batch scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala new file mode 100644 index 000000000000..ede561cbd6b1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenEmptyInSuite extends EmptyInSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala new file mode 100644 index 000000000000..0dd285c7426a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenExpressionsSchemaSuite extends ExpressionsSchemaSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala new file mode 100644 index 000000000000..3c3b438f3cf0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenExtraStrategiesSuite extends ExtraStrategiesSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala new file mode 100644 index 000000000000..9ae4a0f063f4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala @@ -0,0 +1,275 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.BatchScanExecTransformer + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.expressions.{GreaterThan, Literal} +import org.apache.spark.sql.execution.FileSourceScanLike +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.functions.rand +import org.apache.spark.sql.internal.SQLConf + +import org.apache.hadoop.fs.Path + +import java.io.FileNotFoundException + +import scala.collection.mutable + +class GlutenFileBasedDataSourceSuite extends FileBasedDataSourceSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + // test data path is jar path, so failed, test code is same with spark + testGluten("Option recursiveFileLookup: disable partition inferring") { + val dataPath = getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/" + "test-data/text-partitioned" + + val df = spark.read + .format("binaryFile") + .option("recursiveFileLookup", true) + .load(dataPath) + + assert(!df.columns.contains("year"), "Expect partition inferring disabled") + val fileList = df.select("path").collect().map(_.getString(0)) + + val expectedFileList = Array( + dataPath + "/year=2014/data.txt", + dataPath + "/year=2015/data.txt" + ).map(path => "file:" + new Path(path).toString) + + assert(fileList.toSet === expectedFileList.toSet) + } + + testGluten("Spark native readers should respect spark.sql.caseSensitive - parquet") { + withTempDir { + dir => + val format = "parquet" + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" + withTable(tableName) { + val end = 5 + val data = spark.range(end).selectExpr("id as A", "id * 2 as b", "id * 3 as B") + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + data.write.format(format).mode("overwrite").save(tableDir) + } + sql(s"CREATE TABLE $tableName (a LONG, b LONG) USING $format LOCATION '$tableDir'") + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer(sql(s"select a from $tableName"), data.select("A")) + checkAnswer(sql(s"select A from $tableName"), data.select("A")) + + // TODO: gluten can catch exception in executor side, but cannot catch SparkException + // in Driver side + // RuntimeException is triggered at executor side, which is then wrapped as + // SparkException at driver side + // val e1 = intercept[SparkException] { + // sql(s"select b from $tableName").collect() + // } + // + // assert( + // e1.getCause.isInstanceOf[RuntimeException] && + // e1.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode """)) + // val e2 = intercept[SparkException] { + // sql(s"select B from $tableName").collect() + // } + // assert( + // e2.getCause.isInstanceOf[RuntimeException] && + // e2.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode""")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer(sql(s"select a from $tableName"), (0 until end).map(_ => Row(null))) + checkAnswer(sql(s"select b from $tableName"), data.select("b")) + } + } + } + } + + testGluten("SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { + Seq(1.0, 0.5).foreach { + compressionFactor => + withSQLConf( + SQLConf.FILE_COMPRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "350") { + withTempPath { + workDir => + // the file size is 504 bytes + val workDirPath = workDir.getAbsolutePath + val data1 = Seq(100, 200, 300, 400).toDF("count") + data1.write.orc(workDirPath + "/data1") + val df1FromFile = spark.read.orc(workDirPath + "/data1") + val data2 = Seq(100, 200, 300, 400).toDF("count") + data2.write.orc(workDirPath + "/data2") + val df2FromFile = spark.read.orc(workDirPath + "/data2") + val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) + if (compressionFactor == 0.5) { + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.nonEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.isEmpty) + } else { + // compressionFactor is 1.0 + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.isEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } + } + } + } + } + + testGluten("SPARK-25237 compute correct input metrics in FileScanRDD") { + // TODO: Test CSV V2 as well after it implements [[SupportsReportStatistics]]. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "csv") { + withTempPath { + p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty() + // plan is different, so metric is different + assert(bytesReads.sum === 7864) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } + } + + testGluten("SPARK-41017: filter pushdown with nondeterministic predicates") { + withTempPath { + path => + val pathStr = path.getCanonicalPath + spark.range(10).write.parquet(pathStr) + Seq("parquet", "").foreach { + useV1SourceList => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1SourceList) { + val scan = spark.read.parquet(pathStr) + val df = scan.where(rand() > 0.5 && $"id" > 5) + val filters = df.queryExecution.executedPlan.collect { + case f: FileSourceScanLike => f.dataFilters + case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters + case b: BatchScanExecTransformer => b.scan.asInstanceOf[FileScan].dataFilters + }.flatten + assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) + } + } + } + } + + Seq("orc", "parquet").foreach { + format => + testQuietly(GLUTEN_TEST + s"Enabling/disabling ignoreMissingFiles using $format") { + def testIgnoreMissingFiles(options: Map[String, String]): Unit = { + withTempDir { + dir => + val basePath = dir.getCanonicalPath + + Seq("0").toDF("a").write.format(format).save(new Path(basePath, "second").toString) + Seq("1").toDF("a").write.format(format).save(new Path(basePath, "fourth").toString) + + val firstPath = new Path(basePath, "first") + val thirdPath = new Path(basePath, "third") + val fs = thirdPath.getFileSystem(spark.sessionState.newHadoopConf()) + Seq("2").toDF("a").write.format(format).save(firstPath.toString) + Seq("3").toDF("a").write.format(format).save(thirdPath.toString) + val files = Seq(firstPath, thirdPath).flatMap { + p => fs.listStatus(p).filter(_.isFile).map(_.getPath) + } + + val df = spark.read + .options(options) + .format(format) + .load( + new Path(basePath, "first").toString, + new Path(basePath, "second").toString, + new Path(basePath, "third").toString, + new Path(basePath, "fourth").toString) + + // Make sure all data files are deleted and can't be opened. + files.foreach(f => fs.delete(f, false)) + assert(fs.delete(thirdPath, true)) + for (f <- files) { + intercept[FileNotFoundException](fs.open(f)) + } + + checkAnswer(df, Seq(Row("0"), Row("1"))) + } + } + + // Test set ignoreMissingFiles via SQL Conf + // Rewrite this test as error msg is different from velox and data Source reader options + // is not supported. + for { + (ignore, options, sqlConf) <- Seq( + // Set via SQL Conf: leave options empty + ("true", Map.empty[String, String], "true"), + ("false", Map.empty[String, String], "false") + ) + sources <- Seq("", format) + } { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> sources, + SQLConf.IGNORE_MISSING_FILES.key -> sqlConf) { + if (ignore.toBoolean) { + testIgnoreMissingFiles(options) + } else { + val exception = intercept[SparkException] { + testIgnoreMissingFiles(options) + } + assert(exception.getMessage().contains("No such file or directory")) + } + } + } + } + } + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala new file mode 100644 index 000000000000..d5885afaee9c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenFileScanSuite extends FileScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala new file mode 100644 index 000000000000..b3d51e802985 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.internal.SQLConf + +class GlutenGeneratorFunctionSuite extends GeneratorFunctionSuite with GlutenSQLTestsTrait { + testGluten("SPARK-45171: Handle evaluated nondeterministic expression") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = sql("select explode(array(rand(0)))") + checkAnswer(df, Row(0.5488135024422883)) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala new file mode 100644 index 000000000000..26e11f84387a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.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 org.apache.spark.sql + +import org.apache.spark.sql.internal.SQLConf + +class GlutenInjectRuntimeFilterSuite extends InjectRuntimeFilterSuite with GlutenSQLTestsBaseTrait { + + testGluten("GLUTEN-9849: bloom filter applied to partition filter") { + withSQLConf( + SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD.key -> "3000", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000" + ) { + assertRewroteWithBloomFilter( + "select * from bf5part join bf2 on " + + "bf5part.f5 = bf2.c2 where bf2.a2 = 67") + } + } + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala new file mode 100644 index 000000000000..0a354a1fc39e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenIntervalFunctionsSuite extends IntervalFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala new file mode 100644 index 000000000000..5ef4056201ed --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.spark.sql + +class GlutenJoinSuite extends JoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + // Below tests are to verify operators, just skip. + "join operator selection", + "broadcasted hash join operator selection", + "broadcasted hash outer join operator selection", + "broadcasted existence join operator selection", + "SPARK-28323: PythonUDF should be able to use in join condition", + "SPARK-28345: PythonUDF predicate should be able to pushdown to join", + "cross join with broadcast", + "test SortMergeJoin output ordering", + "SPARK-22445 Respect stream-side child's needCopyResult in BroadcastHashJoin", + "SPARK-32330: Preserve shuffled hash join build side partitioning", + "SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering", + "SPARK-32399: Full outer shuffled hash join", + "SPARK-32649: Optimize BHJ/SHJ inner/semi join with empty hashed relation", + "SPARK-34593: Preserve broadcast nested loop join partitioning and ordering", + "SPARK-35984: Config to force applying shuffled hash join", + "test SortMergeJoin (with spill)", + "SPARK-36612: Support left outer join build left or right" + + " outer join build right in shuffled hash join", + // NaN is not supported currently, just skip. + "NaN and -0.0 in join keys" + ) + + testGluten("test case sensitive for BHJ") { + spark.sql("create table t_bhj(a int, b int, C int) using parquet") + spark.sql("insert overwrite t_bhj select id as a, (id+1) as b, (id+2) as c from range(3)") + val sql = + """ + |select /*+ BROADCAST(t1) */ t0.a, t0.b + |from t_bhj as t0 join t_bhj as t1 on t0.a = t1.a and t0.b = t1.b and t0.c = t1.c + |group by t0.a, t0.b + |order by t0.a, t0.b + |""".stripMargin + checkAnswer(spark.sql(sql), Seq(Row(0, 1), Row(1, 2), Row(2, 3))) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala new file mode 100644 index 000000000000..5a28031b6c7a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala @@ -0,0 +1,104 @@ +/* + * 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.spark.sql + +class GlutenJsonFunctionsSuite extends JsonFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("SPARK-42782: Hive compatibility check for get_json_object ") { + val book0 = "{\"author\":\"Nigel Rees\",\"title\":\"Sayings of the Century\"" + + ",\"category\":\"reference\",\"price\":8.95}" + val backet0 = "[1,2,{\"b\":\"y\",\"a\":\"x\"}]" + val backet = "[" + backet0 + ",[3,4],[5,6]]" + val backetFlat = backet0.substring(0, backet0.length() - 1) + ",3,4,5,6]" + + val book = "[" + book0 + ",{\"author\":\"Herman Melville\",\"title\":\"Moby Dick\"," + + "\"category\":\"fiction\",\"price\":8.99" + + ",\"isbn\":\"0-553-21311-3\"},{\"author\":\"J. R. R. Tolkien\"" + + ",\"title\":\"The Lord of the Rings\",\"category\":\"fiction\"" + + ",\"reader\":[{\"age\":25,\"name\":\"bob\"},{\"age\":26,\"name\":\"jack\"}]" + + ",\"price\":22.99,\"isbn\":\"0-395-19395-8\"}]" + + val json = "{\"store\":{\"fruit\":[{\"weight\":8,\"type\":\"apple\"}," + + "{\"weight\":9,\"type\":\"pear\"}],\"basket\":" + backet + ",\"book\":" + book + + ",\"bicycle\":{\"price\":19.95,\"color\":\"red\"}}" + + ",\"email\":\"amy@only_for_json_udf_test.net\"" + + ",\"owner\":\"amy\",\"zip code\":\"94025\",\"fb:testid\":\"1234\"}" + + // Basic test + runTest(json, "$.owner", "amy") + runTest(json, "$.store.bicycle", "{\"price\":19.95,\"color\":\"red\"}") + runTest(json, "$.store.book", book) + runTest(json, "$.store.book[0]", book0) + // runTest(json, "$.store.book[*]", book) - not supported in velox + runTest(json, "$.store.book[0].category", "reference") + // runTest(json, "$.store.book[*].category", + // "[\"reference\",\"fiction\",\"fiction\"]") - not supported in velox + // runTest(json, "$.store.book[*].reader[0].age", "25") - not supported in velox + // runTest(json, "$.store.book[*].reader[*].age", "[25,26]") - not supported in velox + runTest(json, "$.store.basket[0][1]", "2") + // runTest(json, "$.store.basket[*]", backet) - not supported in velox + // runTest(json, "$.store.basket[*][0]", "[1,3,5]") - not supported in velox + // runTest(json, "$.store.basket[0][*]", backet0) - not supported in velox + // runTest(json, "$.store.basket[*][*]", backetFlat) - not supported in velox + runTest(json, "$.store.basket[0][2].b", "y") + // runTest(json, "$.store.basket[0][*].b", "[\"y\"]") - not supported in velox + runTest(json, "$.non_exist_key", null) + runTest(json, "$.store.book[10]", null) + runTest(json, "$.store.book[0].non_exist_key", null) + // runTest(json, "$.store.basket[*].non_exist_key", null) - not supported in velox + // runTest(json, "$.store.basket[0][*].non_exist_key", null) - not supported in velox + // runTest(json, "$.store.basket[*][*].non_exist_key", null) - not supported in velox + runTest(json, "$.zip code", "94025") + runTest(json, "$.fb:testid", "1234") + // runTest("{\"a\":\"b\nc\"}", "$.a", "b\nc") - not supported in velox + + // Test root array + runTest("[1,2,3]", "$[0]", "1") + runTest("[1,2,3]", "$.[0]", null) // Not supported in spark and velox + runTest("[1,2,3]", "$.[1]", null) // Not supported in spark and velox + runTest("[1,2,3]", "$[1]", "2") + + runTest("[1,2,3]", "$[3]", null) + runTest("[1,2,3]", "$.[*]", null) // Not supported in spark and velox + // runTest("[1,2,3]", "$[*]", "[1,2,3]") - not supported in velox + // runTest("[1,2,3]", "$", "[1,2,3]") - not supported in velox + runTest("[{\"k1\":\"v1\"},{\"k2\":\"v2\"},{\"k3\":\"v3\"}]", "$[2]", "{\"k3\":\"v3\"}") + runTest("[{\"k1\":\"v1\"},{\"k2\":\"v2\"},{\"k3\":\"v3\"}]", "$[2].k3", "v3") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0].k11[1]", "2") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0].k11", "[1,2,3]") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0]", "{\"k11\":[1,2,3]}") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1", "[{\"k11\":[1,2,3]}]") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0]", "{\"k1\":[{\"k11\":[1,2,3]}]}") + runTest("[[1,2,3],[4,5,6],[7,8,9]]", "$[1]", "[4,5,6]") + runTest("[[1,2,3],[4,5,6],[7,8,9]]", "$[1][0]", "4") + runTest("[\"a\",\"b\"]", "$[1]", "b") + runTest("[[\"a\",\"b\"]]", "$[0][1]", "b") + + runTest("[1,2,3]", "[0]", null) + runTest("[1,2,3]", "$0", null) + runTest("[1,2,3]", "0", null) + runTest("[1,2,3]", "$.", null) + + runTest("[1,2,3]", "$", "[1,2,3]") + runTest("{\"a\":4}", "$", "{\"a\":4}") + + def runTest(json: String, path: String, exp: String): Unit = { + checkAnswer(Seq(json).toDF().selectExpr(s"get_json_object(value, '$path')"), Row(exp)) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala new file mode 100644 index 000000000000..27cda1c323af --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala @@ -0,0 +1,20 @@ +/* + * 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.spark.sql + +// TODO: fix in Spark-4.0 change back to LateralColumnAliasSuite +class GlutenLateralColumnAliasSuite extends LateralColumnAliasSuiteBase with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala new file mode 100644 index 000000000000..ee39f0138504 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenMathFunctionsSuite extends MathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala new file mode 100644 index 000000000000..d9fc6fd05e1b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenMetadataCacheSuite extends MetadataCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala new file mode 100644 index 000000000000..a95d8a2b2e5c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenMiscFunctionsSuite extends MiscFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala new file mode 100644 index 000000000000..d139221f631a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenNestedDataSourceV1Suite extends NestedDataSourceV1Suite with GlutenSQLTestsTrait {} + +class GlutenNestedDataSourceV2Suite extends NestedDataSourceV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala new file mode 100644 index 000000000000..0887a7416fd8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenParametersSuite extends ParametersSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala new file mode 100644 index 000000000000..f8ab9b16adf4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenProcessingTimeSuite extends ProcessingTimeSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala new file mode 100644 index 000000000000..9cb35efbfbd3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenProductAggSuite extends ProductAggSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala new file mode 100644 index 000000000000..e345309ab114 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenReplaceNullWithFalseInPredicateEndToEndSuite + extends ReplaceNullWithFalseInPredicateEndToEndSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala new file mode 100644 index 000000000000..2d1570be26c5 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenResolveDefaultColumnsSuite + extends ResolveDefaultColumnsSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala new file mode 100644 index 000000000000..abd997bea82e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenRuntimeNullChecksV2Writes extends RuntimeNullChecksV2Writes with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala new file mode 100644 index 000000000000..44a370cbc374 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql + +class GlutenFileSourceSQLInsertTestSuite + extends FileSourceSQLInsertTestSuite + with GlutenSQLTestsTrait {} + +class GlutenDSV2SQLInsertTestSuite extends DSV2SQLInsertTestSuite with GlutenTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..8f397c517ef1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala @@ -0,0 +1,157 @@ +/* + * 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.spark.sql + +import org.apache.spark.SparkException +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.internal.SQLConf + +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("SPARK-28156: self-join should not miss cached view") { + withTable("table1") { + withView("table1_vw") { + withTempView("cachedview") { + val df = Seq.tabulate(5)(x => (x, x + 1, x + 2, x + 3)).toDF("a", "b", "c", "d") + df.write.mode("overwrite").format("parquet").saveAsTable("table1") + sql("drop view if exists table1_vw") + sql("create view table1_vw as select * from table1") + + val cachedView = sql("select a, b, c, d from table1_vw") + + cachedView.createOrReplaceTempView("cachedview") + cachedView.persist() + + val queryDf = sql(s"""select leftside.a, leftside.b + |from cachedview leftside + |join cachedview rightside + |on leftside.a = rightside.a + """.stripMargin) + + val inMemoryTableScan = collect(queryDf.queryExecution.executedPlan) { + case i: InMemoryTableScanExec => i + } + assert(inMemoryTableScan.size == 2) + checkAnswer(queryDf, Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(3, 4) :: Row(4, 5) :: Nil) + } + } + } + + } + + // Velox throw exception : An unsupported nested encoding was found. + ignore( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-33338: GROUP BY using literal map should not fail") { + withTable("t") { + withTempDir { + dir => + sql( + s"CREATE TABLE t USING PARQUET LOCATION '${dir.toURI}' AS SELECT map('k1', 'v1') m," + + s" 'k1' k") + Seq( + "SELECT map('k1', 'v1')[k] FROM t GROUP BY 1", + "SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k]", + "SELECT map('k1', 'v1')[k] a FROM t GROUP BY a" + ).foreach(statement => checkAnswer(sql(statement), Row("v1"))) + } + } + } + + testGluten("SPARK-33593: Vector reader got incorrect data with binary partition value") { + Seq("false").foreach( + value => { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t1") { + sql("""CREATE TABLE t1(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t1"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t2") { + sql("""CREATE TABLE t2(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t2 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t2"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + }) + } + + testGluten( + "SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") { + withTempView("df") { + Seq("m@ca").toDF("s").createOrReplaceTempView("df") + + val e = intercept[SparkException] { + sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() + } + assert( + e.getMessage.contains( + "Escape character must be followed by '%', '_' or the escape character itself")) + + checkAnswer(sql("SELECT s LIKE 'm@@ca' ESCAPE '@' FROM df"), Row(true)) + } + } + + testGluten("the escape character is not allowed to end with") { + withTempView("df") { + Seq("jialiuping").toDF("a").createOrReplaceTempView("df") + + val e = intercept[SparkException] { + sql("SELECT a LIKE 'jialiuping%' ESCAPE '%' FROM df").collect() + } + assert( + e.getMessage.contains( + "Escape character must be followed by '%', '_' or the escape character itself")) + } + } + + ignoreGluten("StreamingQueryProgress.numInputRows should be correct") { + withTempDir { + dir => + val path = dir.toURI.getPath + val numRows = 20 + val df = spark.range(0, numRows) + df.write.mode("overwrite").format("parquet").save(path) + val q = spark.readStream + .format("parquet") + .schema(df.schema) + .load(path) + .writeStream + .format("memory") + .queryName("test") + .start() + q.processAllAvailable + val inputOutputPairs = q.recentProgress.map(p => (p.numInputRows, p.sink.numOutputRows)) + + // numInputRows and sink.numOutputRows must be the same + assert(inputOutputPairs.forall(x => x._1 == x._2)) + + // Sum of numInputRows must match the total number of rows of the input + assert(inputOutputPairs.map(_._1).sum == numRows) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala new file mode 100644 index 000000000000..75bc845b5c8f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenScalaReflectionRelationSuite + extends ScalaReflectionRelationSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala new file mode 100644 index 000000000000..569de43a75ca --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenSerializationSuite extends SerializationSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala new file mode 100644 index 000000000000..ae9b3901afb9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -0,0 +1,55 @@ +/* + * 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.spark.sql + +import org.apache.gluten.config.GlutenConfig + +class GlutenSparkSessionExtensionSuite + extends SparkSessionExtensionSuite + with GlutenTestsCommonTrait { + + testGluten("customColumnarOp") { + val extensions = DummyFilterColmnarHelper.create { + extensions => extensions.injectPlannerStrategy(_ => DummyFilterColumnarStrategy) + } + DummyFilterColmnarHelper.withSession(extensions) { + session => + try { + session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") + def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) + val df = session.sql("SELECT max(id) FROM a") + val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( + session, + df.queryExecution.optimizedPlan) + val result = newDf.collect + newDf.explain(true) + assert(result(0).getLong(0) == 1) + } + testWithFallbackSettings(true, true) + testWithFallbackSettings(true, false) + testWithFallbackSettings(false, true) + testWithFallbackSettings(false, false) + } finally { + session.sql(s"DROP TABLE IF EXISTS a") + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala new file mode 100644 index 000000000000..fab706380488 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala @@ -0,0 +1,76 @@ +/* + * 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.spark.sql + +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC +import org.apache.spark.sql.functions.timestamp_seconds +import org.apache.spark.sql.types.{DataType, DateType, TimestampType} + +import java.util.TimeZone +import java.util.concurrent.TimeUnit + +class GlutenStatisticsCollectionSuite extends StatisticsCollectionSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("store and retrieve column stats in different time zones") { + // TODO: bug fix on TableScan. + // val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) + val (start, end) = (0, 200) + + def checkTimestampStats(t: DataType, srcTimeZone: TimeZone, dstTimeZone: TimeZone)( + checker: ColumnStat => Unit): Unit = { + val table = "time_table" + val column = "T" + val original = TimeZone.getDefault + try { + withTable(table) { + TimeZone.setDefault(srcTimeZone) + spark + .range(start, end) + .select(timestamp_seconds($"id").cast(t).as(column)) + .write + .saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") + + TimeZone.setDefault(dstTimeZone) + val stats = getCatalogTable(table).stats.get.colStats(column).toPlanStat(column, t) + checker(stats) + } + } finally { + TimeZone.setDefault(original) + } + } + + DateTimeTestUtils.outstandingZoneIds.foreach { + zid => + val timeZone = TimeZone.getTimeZone(zid) + checkTimestampStats(DateType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) + assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) + } + checkTimestampStats(TimestampType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) + assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala new file mode 100644 index 000000000000..3d82e214f031 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala @@ -0,0 +1,68 @@ +/* + * 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.spark.sql + +import org.apache.gluten.test.FallbackUtil + +import org.apache.spark.sql.catalyst.expressions.ExpressionEvalHelper +import org.apache.spark.sql.functions._ + +import org.junit.Assert + +class GlutenStringFunctionsSuite + extends StringFunctionsSuite + with GlutenSQLTestsTrait + with ExpressionEvalHelper { + + import testImplicits._ + + testGluten("string split function with no limit and regex pattern") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A")) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_")) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + testGluten("string split function with limit explicitly set to 0") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", 0)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", 0)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + testGluten("string split function with negative limit") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", -1)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", -2)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala new file mode 100644 index 000000000000..323c5fbe1477 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenSubqueryHintPropagationSuite + extends SubqueryHintPropagationSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala new file mode 100644 index 000000000000..05dae6290214 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala @@ -0,0 +1,55 @@ +/* + * 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.spark.sql + +import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} + +class GlutenSubquerySuite extends SubquerySuite with GlutenSQLTestsTrait { + + // Test Canceled: IntegratedUDFTestUtils.shouldTestPythonUDFs was false + override def testNameBlackList: Seq[String] = Seq( + "SPARK-28441: COUNT bug in WHERE clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug in SELECT clause (Project) with PythonUDF", + "SPARK-28441: COUNT bug in Aggregate with PythonUDF", + "SPARK-28441: COUNT bug negative examples with PythonUDF", + "SPARK-28441: COUNT bug in nested subquery with PythonUDF", + "SPARK-28441: COUNT bug with nasty predicate expr with PythonUDF", + "SPARK-28441: COUNT bug in HAVING clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug with attribute ref in subquery input and output with PythonUDF" + ) + + // === Following cases override super class's cases === + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-26893 Allow pushdown of partition pruning subquery filters to file source") { + withTable("a", "b") { + spark.range(4).selectExpr("id", "id % 2 AS p").write.partitionBy("p").saveAsTable("a") + spark.range(2).write.saveAsTable("b") + + // need to execute the query before we can examine fs.inputRDDs() + val df = sql("SELECT * FROM a WHERE p <= (SELECT MIN(id) FROM b)") + checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) + assert(stripAQEPlan(df.queryExecution.executedPlan).collectFirst { + case t: WholeStageTransformer => t + } match { + case Some(WholeStageTransformer(fs: FileSourceScanExecTransformer, _)) => + fs.dynamicallySelectedPartitions.toPartitionArray + .exists(_.filePath.toString.contains("p=0")) + case _ => false + }) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala new file mode 100644 index 000000000000..35858433b406 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenTableOptionsConstantFoldingSuite + extends TableOptionsConstantFoldingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala new file mode 100644 index 000000000000..cff309cfce2b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql + +class GlutenTypedImperativeAggregateSuite + extends TypedImperativeAggregateSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala new file mode 100644 index 000000000000..2dcde94c13cd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala @@ -0,0 +1,58 @@ +/* + * 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.spark.sql + +class GlutenUnwrapCastInComparisonEndToEndSuite + extends UnwrapCastInComparisonEndToEndSuite + with GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("cases when literal is max") { + withTable(t) { + Seq[(Integer, java.lang.Short, java.lang.Float)]( + (1, 100.toShort, 3.14.toFloat), + (2, Short.MaxValue, Float.NaN), + (3, Short.MinValue, Float.PositiveInfinity), + (4, 0.toShort, Float.MaxValue), + (5, null, null)) + .toDF("c1", "c2", "c3") + .write + .saveAsTable(t) + val df = spark.table(t) + + val lit = Short.MaxValue.toInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + + // NaN is not supported in velox, so unexpected result will be obtained. +// checkAnswer(df.where(s"c3 > double('nan')").select("c1"), Seq.empty) +// checkAnswer(df.where(s"c3 >= double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 == double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 <=> double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 != double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) +// checkAnswer(df.where(s"c3 <= double('nan')").select("c1"), +// Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) +// checkAnswer(df.where(s"c3 < double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala new file mode 100644 index 000000000000..ae173ecd47f9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenUrlFunctionsSuite extends UrlFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala new file mode 100644 index 000000000000..918a96c49e30 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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.spark.sql + +class GlutenXPathFunctionsSuite extends XPathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala new file mode 100644 index 000000000000..14079037518f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenArithmeticExpressionSuite extends ArithmeticExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala new file mode 100644 index 000000000000..fd9827ddf502 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenBitwiseExpressionsSuite extends BitwiseExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala new file mode 100644 index 000000000000..ef06a9853973 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala @@ -0,0 +1,291 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, ALL_TIMEZONES, UTC, UTC_OPT} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{fromJavaTimestamp, millisToMicros, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.util.DebuggableThreadUtils + +import java.sql.{Date, Timestamp} +import java.util.{Calendar, TimeZone} + +class GlutenCastSuite extends CastWithAnsiOffSuite with GlutenTestsTrait { + + override def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): Cast = { + v match { + case lit: Expression => + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + case _ => + val lit = Literal(v) + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + } + } + + // Register UDT For test("SPARK-32828") + UDTRegistration.register(classOf[IExampleBaseType].getName, classOf[ExampleBaseTypeUDT].getName) + UDTRegistration.register(classOf[IExampleSubType].getName, classOf[ExampleSubTypeUDT].getName) + + testGluten("missing cases - from boolean") { + (DataTypeTestUtils.numericTypeWithoutDecimal ++ Set(BooleanType)).foreach { + t => + t match { + case BooleanType => + checkEvaluation(cast(cast(true, BooleanType), t), true) + checkEvaluation(cast(cast(false, BooleanType), t), false) + case _ => + checkEvaluation(cast(cast(true, BooleanType), t), 1) + checkEvaluation(cast(cast(false, BooleanType), t), 0) + } + } + } + + testGluten("missing cases - from byte") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ByteType), t), 0) + checkEvaluation(cast(cast(-1, ByteType), t), -1) + checkEvaluation(cast(cast(1, ByteType), t), 1) + } + } + + testGluten("missing cases - from short") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ShortType), t), 0) + checkEvaluation(cast(cast(-1, ShortType), t), -1) + checkEvaluation(cast(cast(1, ShortType), t), 1) + } + } + + testGluten("missing cases - date self check") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(cast(d, DateType), d) + } + + testGluten("data type casting") { + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts)) + + // SystemV timezones are a legacy way of specifying timezones in Unix-like OS. + // It is not supported by Velox. + for (tz <- ALL_TIMEZONES.filterNot(_.getId.contains("SystemV"))) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz.getId + ) { + val timeZoneId = Option(tz.getId) + var c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 2, 8, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 10, 1, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + } + } + + checkEvaluation(cast("abdef", StringType), "abdef") + checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) + + checkEvaluation(cast(cast(sd, DateType), StringType), sd) + checkEvaluation(cast(cast(d, StringType), DateType), 0) + + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get + ) { + checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) + checkEvaluation( + cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), + fromJavaTimestamp(ts)) + + // all convert to string type to check + checkEvaluation( + cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType), + sd) + checkEvaluation( + cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT), + zts) + } + + checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") + + checkEvaluation( + cast( + cast(cast(cast(cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), + LongType), + 5.toLong) + + checkEvaluation(cast("23", DoubleType), 23d) + checkEvaluation(cast("23", IntegerType), 23) + checkEvaluation(cast("23", FloatType), 23f) + checkEvaluation(cast("23", DecimalType.USER_DEFAULT), Decimal(23)) + checkEvaluation(cast("23", ByteType), 23.toByte) + checkEvaluation(cast("23", ShortType), 23.toShort) + checkEvaluation(cast(123, IntegerType), 123) + + checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) + } + + test("cast from boolean to timestamp") { + val tsTrue = new Timestamp(0) + tsTrue.setNanos(1000) + + val tsFalse = new Timestamp(0) + + checkEvaluation(cast(true, TimestampType), tsTrue) + + checkEvaluation(cast(false, TimestampType), tsFalse) + } + + testGluten("cast string to timestamp") { + DebuggableThreadUtils.parmap( + ALL_TIMEZONES + .filterNot(_.getId.contains("SystemV")) + .filterNot(_.getId.contains("Europe/Kyiv")) + .filterNot(_.getId.contains("America/Ciudad_Juarez")) + .filterNot(_.getId.contains("Antarctica/Vostok")) + .filterNot(_.getId.contains("Pacific/Kanton")) + .filterNot(_.getId.contains("Asia/Tehran")) + .filterNot(_.getId.contains("Iran")), + prefix = "CastSuiteBase-cast-string-to-timestamp", + maxThreads = 1 + ) { + zid => + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { + checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) + } + + val tz = TimeZone.getTimeZone(zid) + var c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 ", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18 12:03:17", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17+7:3", + // new Timestamp(c.getTimeInMillis)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18 12:03:17.123", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123-1:0", + // new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123-01:00", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123+07:30", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123+7:3", + // new Timestamp(c.getTimeInMillis)) + } + } + } + + testGluten("cast decimal to timestamp") { + val tz = TimeZone.getTimeZone(TimeZone.getDefault.getID) + val c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 123) + val d = Decimal(c.getTimeInMillis.toDouble / 1000) + checkEvaluation(cast(d, TimestampType), new Timestamp(c.getTimeInMillis)) + } + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala new file mode 100644 index 000000000000..ca6ef9df0f48 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.types._ + +import scala.util.Random + +class GlutenCollectionExpressionsSuite extends CollectionExpressionsSuite with GlutenTestsTrait { + testGluten("Shuffle") { + // Primitive-type elements + val ai0 = Literal.create(Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, containsNull = false)) + val ai1 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) + val ai2 = Literal.create(Seq(null, 1, null, 3), ArrayType(IntegerType, containsNull = true)) + val ai3 = Literal.create(Seq(2, null, 4, null), ArrayType(IntegerType, containsNull = true)) + val ai4 = Literal.create(Seq(null, null, null), ArrayType(IntegerType, containsNull = true)) + val ai5 = Literal.create(Seq(1), ArrayType(IntegerType, containsNull = false)) + val ai6 = Literal.create(Seq.empty, ArrayType(IntegerType, containsNull = false)) + val ai7 = Literal.create(null, ArrayType(IntegerType, containsNull = true)) + + checkEvaluation(Shuffle(ai0, Some(0)), Array(2, 1, 5, 4, 3)) + checkEvaluation(Shuffle(ai1, Some(0)), Array(2, 1, 3)) + checkEvaluation(Shuffle(ai2, Some(0)), Array(1, null, null, 3)) + checkEvaluation(Shuffle(ai3, Some(0)), Array(null, 2, 4, null)) + checkEvaluation(Shuffle(ai4, Some(0)), Array(null, null, null)) + checkEvaluation(Shuffle(ai5, Some(0)), Array(1)) + checkEvaluation(Shuffle(ai6, Some(0)), Array.empty) + checkEvaluation(Shuffle(ai7, Some(0)), null) + + // Non-primitive-type elements + val as0 = Literal.create(Seq("a", "b", "c", "d"), ArrayType(StringType, containsNull = false)) + val as1 = Literal.create(Seq("a", "b", "c"), ArrayType(StringType, containsNull = false)) + val as2 = Literal.create(Seq(null, "a", null, "c"), ArrayType(StringType, containsNull = true)) + val as3 = Literal.create(Seq("b", null, "d", null), ArrayType(StringType, containsNull = true)) + val as4 = Literal.create(Seq(null, null, null), ArrayType(StringType, containsNull = true)) + val as5 = Literal.create(Seq("a"), ArrayType(StringType, containsNull = false)) + val as6 = Literal.create(Seq.empty, ArrayType(StringType, containsNull = false)) + val as7 = Literal.create(null, ArrayType(StringType, containsNull = true)) + val aa = + Literal.create(Seq(Seq("a", "b"), Seq("c", "d"), Seq("e")), ArrayType(ArrayType(StringType))) + + checkEvaluation(Shuffle(as0, Some(0)), Array("b", "a", "c", "d")) + checkEvaluation(Shuffle(as1, Some(0)), Array("b", "a", "c")) + checkEvaluation(Shuffle(as2, Some(0)), Array("a", null, null, "c")) + checkEvaluation(Shuffle(as3, Some(0)), Array(null, "b", "d", null)) + checkEvaluation(Shuffle(as4, Some(0)), Array(null, null, null)) + checkEvaluation(Shuffle(as5, Some(0)), Array("a")) + checkEvaluation(Shuffle(as6, Some(0)), Array.empty) + checkEvaluation(Shuffle(as7, Some(0)), null) + checkEvaluation(Shuffle(aa, Some(0)), Array(Array("c", "d"), Array("a", "b"), Array("e"))) + + val r = new Random(1234) + val seed1 = Some(r.nextLong()) + assert( + evaluateWithoutCodegen(Shuffle(ai0, seed1)) === + evaluateWithoutCodegen(Shuffle(ai0, seed1))) + assert( + evaluateWithMutableProjection(Shuffle(ai0, seed1)) === + evaluateWithMutableProjection(Shuffle(ai0, seed1))) + assert( + evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) === + evaluateWithUnsafeProjection(Shuffle(ai0, seed1))) + + val seed2 = Some(r.nextLong()) + assert( + evaluateWithoutCodegen(Shuffle(ai0, seed1)) !== + evaluateWithoutCodegen(Shuffle(ai0, seed2))) + assert( + evaluateWithMutableProjection(Shuffle(ai0, seed1)) !== + evaluateWithMutableProjection(Shuffle(ai0, seed2))) + assert( + evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) !== + evaluateWithUnsafeProjection(Shuffle(ai0, seed2))) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala new file mode 100644 index 000000000000..f5f278361e1f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenComplexTypeSuite extends ComplexTypeSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala new file mode 100644 index 000000000000..923f5f87bcc2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenConditionalExpressionSuite extends ConditionalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala new file mode 100644 index 000000000000..704e4f5d39c5 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala @@ -0,0 +1,663 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.time.{LocalDateTime, ZoneId} +import java.util.{Calendar, Locale, TimeZone} +import java.util.concurrent.TimeUnit._ + +class GlutenDateExpressionsSuite extends DateExpressionsSuite with GlutenTestsTrait { + override def testIntegralInput(testFunc: Number => Unit): Unit = { + def checkResult(input: Long): Unit = { + if (input.toByte == input) { + testFunc(input.toByte) + } else if (input.toShort == input) { + testFunc(input.toShort) + } else if (input.toInt == input) { + testFunc(input.toInt) + } else { + testFunc(input) + } + } + + checkResult(0) + checkResult(Byte.MaxValue) + checkResult(Byte.MinValue) + checkResult(Short.MaxValue) + checkResult(Short.MinValue) + // Spark collect causes integer overflow. + // checkResult(Int.MaxValue) + // checkResult(Int.MinValue) + // checkResult(Int.MaxValue.toLong + 100) + // checkResult(Int.MinValue.toLong - 100) + } + + testGluten("TIMESTAMP_MICROS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation(MicrosToTimestamp(Literal(value)), value.longValue()) + } + + // test null input + checkEvaluation(MicrosToTimestamp(Literal(null, IntegerType)), null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test max/min input + // Spark collect causes long overflow. + // testIntegralFunc(Long.MaxValue) + // testIntegralFunc(Long.MinValue) + } + + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like UTC. + // "UTC", + PST.getId, + CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels") + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + + // TODO: fix in Spark-4.0 + ignoreGluten("unix_timestamp") { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 0L) + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal(new Timestamp(1000000)), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + checkEvaluation( + UnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + UnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), + tz.toZoneId)) + ) + val t1 = UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + .eval() + .asInstanceOf[Long] + val t2 = UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + .eval() + .asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + UnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + // TODO: fix in Spark-4.0 + ignoreGluten("to_unix_timestamp") { + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), + 0L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal(fmt1), + timeZoneId), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(new Timestamp(1000000)), Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds(DateTimeUtils + .daysToMicros(DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), zid)) + ) + val t1 = ToUnixTimestamp(CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + val t2 = ToUnixTimestamp(CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + ToUnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal.create(null, DateType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid)) + ) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = ToUnixTimestamp( + BoundReference(ordinal = 0, dataType = StringType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = 0L, + inputRow = InternalRow( + UTF8String.fromString(sdf1.format(new Timestamp(0))), + UTF8String.fromString(fmt1)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + // Modified based on vanilla spark to explicitly set timezone in config. + testGluten("DateFormat") { + val PST_OPT = Option("America/Los_Angeles") + val JST_OPT = Option("Asia/Tokyo") + + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), + null) + checkEvaluation( + DateFormatClass( + Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), + UTC_OPT), + null) + + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("y"), UTC_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("H"), UTC_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("y"), PST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("H"), PST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> JST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("y"), JST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("H"), JST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + } + } + } + + testGluten("from_unixtime") { + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like "UTC". + // "UTC", + // Not supported in velox. + // PST.getId, + // CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels" + ) + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + checkEvaluation( + FromUnixTime(Literal(0L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(0))) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(1000000))) + checkEvaluation( + FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000000))) + checkEvaluation( + FromUnixTime(Literal(Long.MaxValue), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000))) + checkEvaluation( + FromUnixTime( + Literal.create(null, LongType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal.create(null, LongType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), + null) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = FromUnixTime( + BoundReference(ordinal = 0, dataType = LongType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = UTF8String.fromString(sdf1.format(new Timestamp(0))), + inputRow = InternalRow(0L, UTF8String.fromString(fmt1)) + ) + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\""), UTC_OPT) :: Nil) + } + + testGluten("Hour") { + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like "UTC". + // "UTC", + // Due to known issue: "-08:00/+01:00 not found in timezone database", + // skip check PST, CET timezone here. + // https://github.com/facebookincubator/velox/issues/7804 + // PST.getId, CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels" + ) + withDefaultTimeZone(UTC) { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy + ) { + assert(Hour(Literal.create(null, DateType), UTC_OPT).resolved === false) + assert(Hour(Literal(ts), UTC_OPT).resolved) + Seq(TimestampType, TimestampNTZType).foreach { + dt => + checkEvaluation(Hour(Cast(Literal(d), dt, UTC_OPT), UTC_OPT), 0) + checkEvaluation(Hour(Cast(Literal(date), dt, UTC_OPT), UTC_OPT), 13) + } + checkEvaluation(Hour(Literal(ts), UTC_OPT), 13) + } + + val c = Calendar.getInstance() + outstandingTimezonesIds.foreach { + zid => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid + ) { + val timeZoneId = Option(zid) + c.setTimeZone(TimeZone.getTimeZone(zid)) + (0 to 24 by 5).foreach { + h => + // validate timestamp with local time zone + c.set(2015, 18, 3, h, 29, 59) + checkEvaluation( + Hour(Literal(new Timestamp(c.getTimeInMillis)), timeZoneId), + c.get(Calendar.HOUR_OF_DAY)) + + // validate timestamp without time zone + val localDateTime = LocalDateTime.of(2015, 1, 3, h, 29, 59) + checkEvaluation(Hour(Literal(localDateTime), timeZoneId), h) + } + Seq(TimestampType, TimestampNTZType).foreach { + dt => + checkConsistencyBetweenInterpretedAndCodegen( + (child: Expression) => Hour(child, timeZoneId), + dt) + } + } + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-42635: timestampadd near daylight saving transition") { + // In America/Los_Angeles timezone, timestamp value `skippedTime` is 2011-03-13 03:00:00. + // The next second of 2011-03-13 01:59:59 jumps to 2011-03-13 03:00:00. + val skippedTime = 1300010400000000L + // In America/Los_Angeles timezone, both timestamp range `[repeatedTime - MICROS_PER_HOUR, + // repeatedTime)` and `[repeatedTime, repeatedTime + MICROS_PER_HOUR)` map to + // [2011-11-06 01:00:00, 2011-11-06 02:00:00). + // The next second of 2011-11-06 01:59:59 (pre-transition) jumps back to 2011-11-06 01:00:00. + val repeatedTime = 1320570000000000L + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId) { + // Adding one day is **not** equivalent to adding _PER_DAY time units, because not every + // day has 24 hours: 2011-03-13 has 23 hours, 2011-11-06 has 25 hours. + + // timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 + checkEvaluation( + TimestampAdd("DAY", Literal(1), Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime) + // timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(24), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR) + // timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(23), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime) + // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 + checkEvaluation( + TimestampAdd( + "SECOND", + Literal(SECONDS_PER_DAY.toInt), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR) + // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 + checkEvaluation( + TimestampAdd( + "SECOND", + Literal(SECONDS_PER_DAY.toInt - 1), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR - MICROS_PER_SECOND + ) + + // timestampadd(DAY, 1, 2011-11-05 02:00:00) = 2011-11-06 02:00:00 + checkEvaluation( + TimestampAdd( + "DAY", + Literal(1), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime + MICROS_PER_HOUR) + // timestampadd(DAY, 1, 2011-11-05 01:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "DAY", + Literal(1), + Literal(repeatedTime - 25 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(DAY, -1, 2011-11-07 01:00:00) = 2011-11-06 01:00:00 (post-transition) + // Vanilla spark result is 1320570000000000L, velox result is 1320566400000000L, they + // are all 2011-11-06 01:00:00. + checkEvaluation( + TimestampAdd( + "DAY", + Literal(-1), + Literal(repeatedTime + 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(MONTH, 1, 2011-10-06 01:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "MONTH", + Literal(1), + Literal(repeatedTime - MICROS_PER_HOUR - 31 * MICROS_PER_DAY, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(MONTH, -1, 2011-12-06 01:00:00) = 2011-11-06 01:00:00 (post-transition) + // Vanilla spark result is 1320570000000000L, velox result is 1320566400000000L, they + // are all 2011-11-06 01:00:00. + checkEvaluation( + TimestampAdd( + "MONTH", + Literal(-1), + Literal(repeatedTime + 30 * MICROS_PER_DAY, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(HOUR, 23, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(23), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(HOUR, 24, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (post-transition) + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(24), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime) + } + } + + testGluten("months_between") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val timeZoneId = Option(zid.getId) + sdf.setTimeZone(TimeZone.getTimeZone(zid)) + + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("1997-02-28 10:30:00").getTime)), + Literal(new Timestamp(sdf.parse("1996-10-30 00:00:00").getTime)), + Literal.TrueLiteral, + timeZoneId = timeZoneId + ), + 3.94959677 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("1997-02-28 10:30:00").getTime)), + Literal(new Timestamp(sdf.parse("1996-10-30 00:00:00").getTime)), + Literal.FalseLiteral, + timeZoneId = timeZoneId + ), + 3.9495967741935485 + ) + + Seq(Literal.FalseLiteral, Literal.TrueLiteral).foreach { + roundOff => + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-30 11:52:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-01-30 11:50:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + 0.0 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-31 00:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + -2.0 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-02-28 00:00:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + 1.0 + ) + } + val t = Literal(Timestamp.valueOf("2015-03-31 22:00:00")) + val tnull = Literal.create(null, TimestampType) + checkEvaluation(MonthsBetween(t, tnull, Literal.TrueLiteral, timeZoneId = timeZoneId), null) + checkEvaluation(MonthsBetween(tnull, t, Literal.TrueLiteral, timeZoneId = timeZoneId), null) + checkEvaluation( + MonthsBetween(tnull, tnull, Literal.TrueLiteral, timeZoneId = timeZoneId), + null) + checkEvaluation( + MonthsBetween(t, t, Literal.create(null, BooleanType), timeZoneId = timeZoneId), + null) + checkConsistencyBetweenInterpretedAndCodegen( + (time1: Expression, time2: Expression, roundOff: Expression) => + MonthsBetween(time1, time2, roundOff, timeZoneId = timeZoneId), + TimestampType, + TimestampType, + BooleanType + ) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala new file mode 100644 index 000000000000..8f9054928e40 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenDecimalExpressionSuite extends DecimalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala new file mode 100644 index 000000000000..97e752d7d046 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala @@ -0,0 +1,138 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.gluten.expression._ + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.types._ + +class GlutenDecimalPrecisionSuite extends GlutenTestsTrait { + private val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry) + private val analyzer = new Analyzer(catalog) + + private val relation = LocalRelation( + AttributeReference("i", IntegerType)(), + AttributeReference("d1", DecimalType(2, 1))(), + AttributeReference("d2", DecimalType(5, 2))(), + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("f", FloatType)(), + AttributeReference("b", DoubleType)() + ) + + private val i: Expression = UnresolvedAttribute("i") + private val d1: Expression = UnresolvedAttribute("d1") + private val d2: Expression = UnresolvedAttribute("d2") + private val u: Expression = UnresolvedAttribute("u") + private val f: Expression = UnresolvedAttribute("f") + private val b: Expression = UnresolvedAttribute("b") + + private def checkType(expression: Expression, expectedType: DataType): Unit = { + val plan = analyzer.execute(Project(Seq(Alias(expression, "c")()), relation)) + assert(plan.isInstanceOf[Project]) + val expr = plan.asInstanceOf[Project].projectList.head + assert(expr.dataType == expectedType) + val transformedExpr = + ExpressionConverter.replaceWithExpressionTransformer(expr, plan.inputSet.toSeq) + assert(transformedExpr.dataType == expectedType) + } + + private def stripAlias(expr: Expression): Expression = { + expr match { + case a: Alias => stripAlias(a.child) + case _ => expr + } + } + + private def checkComparison(expression: Expression, expectedType: DataType): Unit = { + val plan = analyzer.execute(Project(Alias(expression, "c")() :: Nil, relation)) + assert(plan.isInstanceOf[Project]) + val expr = stripAlias(plan.asInstanceOf[Project].projectList.head) + val transformedExpr = + ExpressionConverter.replaceWithExpressionTransformer(expr, plan.inputSet.toSeq) + assert(transformedExpr.isInstanceOf[GenericExpressionTransformer]) + val binaryComparison = transformedExpr.asInstanceOf[GenericExpressionTransformer] + assert(binaryComparison.original.isInstanceOf[BinaryComparison]) + assert(binaryComparison.children.size == 2) + assert(binaryComparison.children.forall(_.dataType == expectedType)) + } + + test("basic operations") { + checkType(Add(d1, d2), DecimalType(6, 2)) + checkType(Subtract(d1, d2), DecimalType(6, 2)) + checkType(Multiply(d1, d2), DecimalType(8, 3)) + checkType(Divide(d1, d2), DecimalType(10, 7)) + checkType(Divide(d2, d1), DecimalType(10, 6)) + + checkType(Add(Add(d1, d2), d1), DecimalType(7, 2)) + checkType(Add(Add(d1, d1), d1), DecimalType(4, 1)) + checkType(Add(d1, Add(d1, d1)), DecimalType(4, 1)) + checkType(Add(Add(Add(d1, d2), d1), d2), DecimalType(8, 2)) + checkType(Add(Add(d1, d2), Add(d1, d2)), DecimalType(7, 2)) + checkType(Subtract(Subtract(d2, d1), d1), DecimalType(7, 2)) + checkType(Multiply(Multiply(d1, d1), d2), DecimalType(11, 4)) + checkType(Divide(d2, Add(d1, d1)), DecimalType(10, 6)) + } + + test("Comparison operations") { + checkComparison(EqualTo(i, d1), DecimalType(11, 1)) + checkComparison(EqualNullSafe(d2, d1), DecimalType(5, 2)) + checkComparison(LessThan(i, d1), DecimalType(11, 1)) + checkComparison(LessThanOrEqual(d1, d2), DecimalType(5, 2)) + checkComparison(GreaterThan(d2, u), DecimalType.SYSTEM_DEFAULT) + checkComparison(GreaterThanOrEqual(d1, f), DoubleType) + checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) + } + + test("bringing in primitive types") { + checkType(Add(d1, i), DecimalType(12, 1)) + checkType(Add(d1, f), DoubleType) + checkType(Add(i, d1), DecimalType(12, 1)) + checkType(Add(f, d1), DoubleType) + checkType(Add(d1, Cast(i, LongType)), DecimalType(22, 1)) + checkType(Add(d1, Cast(i, ShortType)), DecimalType(7, 1)) + checkType(Add(d1, Cast(i, ByteType)), DecimalType(5, 1)) + checkType(Add(d1, Cast(i, DoubleType)), DoubleType) + } + + test("maximum decimals") { + for (expr <- Seq(d1, d2, i, u)) { + checkType(Add(expr, u), DecimalType(38, 17)) + checkType(Subtract(expr, u), DecimalType(38, 17)) + } + + checkType(Multiply(d1, u), DecimalType(38, 16)) + checkType(Multiply(d2, u), DecimalType(38, 14)) + checkType(Multiply(i, u), DecimalType(38, 7)) + checkType(Multiply(u, u), DecimalType(38, 6)) + + checkType(Divide(u, d1), DecimalType(38, 17)) + checkType(Divide(u, d2), DecimalType(38, 16)) + checkType(Divide(u, i), DecimalType(38, 18)) + checkType(Divide(u, u), DecimalType(38, 6)) + + for (expr <- Seq(f, b)) { + checkType(Add(expr, u), DoubleType) + checkType(Subtract(expr, u), DoubleType) + checkType(Multiply(expr, u), DoubleType) + checkType(Divide(expr, u), DoubleType) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala new file mode 100644 index 000000000000..d1867936c149 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenGeneratorExpressionSuite extends GeneratorExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala new file mode 100644 index 000000000000..4f9d1ffff271 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenHashExpressionsSuite extends HashExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala new file mode 100644 index 000000000000..6687e707924b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenHigherOrderFunctionsSuite extends HigherOrderFunctionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala new file mode 100644 index 000000000000..2b8aec03d7bd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenIntervalExpressionsSuite extends IntervalExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala new file mode 100644 index 000000000000..f9d314e508e9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenJsonExpressionsSuite extends JsonExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala new file mode 100644 index 000000000000..556d185af078 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenLiteralExpressionSuite extends LiteralExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala new file mode 100644 index 000000000000..d49bbd3555eb --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala @@ -0,0 +1,280 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ + +class GlutenMathExpressionsSuite extends MathExpressionsSuite with GlutenTestsTrait { + testGluten("round/bround/floor/ceil") { + val scales = -6 to 6 + val doublePi: Double = math.Pi + val shortPi: Short = 31415 + val intPi: Int = 314159265 + val longPi: Long = 31415926535897932L + val bdPi: BigDecimal = BigDecimal(31415927L, 7) + val floatPi: Float = 3.1415f + + val doubleResults: Seq[Double] = + Seq(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 3.0, 3.1, 3.14, 3.142, 3.1416, 3.14159, 3.141593) + + val floatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.142f, 3.1415f, 3.1415f, 3.1415f) + + val bRoundFloatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.141f, 3.1415f, 3.1415f, 3.1415f) + + val shortResults: Seq[Short] = Seq[Short](0, 0, 30000, 31000, 31400, 31420) ++ + Seq.fill[Short](7)(31415) + + val intResults: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + + val longResults: Seq[Long] = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535897900L, 31415926535897930L) ++ + Seq.fill(7)(31415926535897932L) + + val intResultsB: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159260) ++ Seq.fill(7)( + 314159265) + + def doubleResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.14159, 3.141592) + Decimal(results(i)) + } + + def doubleResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1416, 3.1416, 3.141593) + Decimal(results(i)) + } + + def floatResultsFloor(i: Int): Any = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def floatResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def shortResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 30000, 31000, 31400, 31410) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def shortResultsCeil(i: Int): Decimal = { + val results = Seq(1000000, 100000, 40000, 32000, 31500, 31420) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def longResultsFloor(i: Int): Decimal = { + val results = Seq(31415926535000000L, 31415926535800000L, 31415926535890000L, + 31415926535897000L, 31415926535897900L, 31415926535897930L, 31415926535897932L) ++ + Seq.fill(6)(31415926535897932L) + Decimal(results(i)) + } + + def longResultsCeil(i: Int): Decimal = { + val results = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535898000L, 31415926535897940L) ++ + Seq.fill(7)(31415926535897932L) + Decimal(results(i)) + } + + def intResultsFloor(i: Int): Decimal = { + val results = + Seq(314000000, 314100000, 314150000, 314159000, 314159200, 314159260) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + def intResultsCeil(i: Int): Decimal = { + val results = + Seq(315000000, 314200000, 314160000, 314160000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + scales.zipWithIndex.foreach { + case (scale, i) => + checkEvaluation(Round(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(Round(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(Round(intPi, scale), intResults(i), EmptyRow) + checkEvaluation(Round(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(Round(floatPi, scale), floatResults(i), EmptyRow) + checkEvaluation(BRound(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(BRound(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(BRound(intPi, scale), intResultsB(i), EmptyRow) + checkEvaluation(BRound(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(BRound(floatPi, scale), bRoundFloatResults(i), EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(doublePi), Literal(scale))), + doubleResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(shortPi), Literal(scale))), + shortResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(intPi), Literal(scale))), + intResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(longPi), Literal(scale))), + longResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(floatPi), Literal(scale))), + floatResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(doublePi), Literal(scale))), + doubleResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(shortPi), Literal(scale))), + shortResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(intPi), Literal(scale))), + intResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(longPi), Literal(scale))), + longResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(floatPi), Literal(scale))), + floatResultsCeil(i), + EmptyRow) + } + + val bdResults: Seq[BigDecimal] = Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14159"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + val bdResultsFloor: Seq[BigDecimal] = + Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.141"), + BigDecimal("3.1415"), + BigDecimal("3.14159"), + BigDecimal("3.141592"), + BigDecimal("3.1415927") + ) + + val bdResultsCeil: Seq[BigDecimal] = Seq( + BigDecimal(4), + BigDecimal("3.2"), + BigDecimal("3.15"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14160"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + (0 to 7).foreach { + i => + checkEvaluation(Round(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(BRound(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(RoundFloor(bdPi, i), bdResultsFloor(i), EmptyRow) + checkEvaluation(RoundCeil(bdPi, i), bdResultsCeil(i), EmptyRow) + } + (8 to 10).foreach { + scale => + checkEvaluation(Round(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(BRound(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundFloor(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundCeil(bdPi, scale), bdPi, EmptyRow) + } + + DataTypeTestUtils.numericTypes.foreach { + dataType => + checkEvaluation(Round(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + Round(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation(BRound(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + BRound(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal.create(null, dataType), Literal(2))), + null) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal.create(null, dataType), Literal(2))), + null) + } + + checkEvaluation(Round(2.5, 0), 3.0) + checkEvaluation(Round(3.5, 0), 4.0) + checkEvaluation(Round(-2.5, 0), -3.0) + checkEvaluation(Round(-3.5, 0), -4.0) + checkEvaluation(Round(-0.35, 1), -0.4) + checkEvaluation(Round(-35, -1), -40) + checkEvaluation(Round(BigDecimal("45.00"), -1), BigDecimal(50)) + checkEvaluation(Round(44, -1), 40) + checkEvaluation(Round(78, 1), 78) + checkEvaluation(BRound(2.5, 0), 2.0) + checkEvaluation(BRound(3.5, 0), 4.0) + checkEvaluation(BRound(-2.5, 0), -2.0) + checkEvaluation(BRound(-3.5, 0), -4.0) + checkEvaluation(BRound(-0.35, 1), -0.4) + checkEvaluation(BRound(-35, -1), -40) + checkEvaluation(Round(1.12345678901234567, 8), 1.12345679) + checkEvaluation(Round(-0.98765432109876543, 5), -0.98765) + checkEvaluation(Round(12345.67890123456789, 6), 12345.678901) + // Enable the test after fixing https://github.com/apache/incubator-gluten/issues/6827 + // checkEvaluation(Round(0.5549999999999999, 2), 0.55) + checkEvaluation(BRound(BigDecimal("45.00"), -1), BigDecimal(40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(2.5), Literal(0))), Decimal(2)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-2.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-3.5), Literal(0))), Decimal(-4L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.35), Literal(1))), Decimal(-0.4)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-35), Literal(-1))), Decimal(-40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.1), Literal(0))), Decimal(-1)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.1411), Literal(-3))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(135.135), Literal(-2))), Decimal(100)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(2.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.5), Literal(0))), Decimal(4L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-2.5), Literal(0))), Decimal(-2L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-3.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.35), Literal(1))), Decimal(-0.3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-35), Literal(-1))), Decimal(-30)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.1), Literal(0))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.1411), Literal(-3))), Decimal(1000)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(135.135), Literal(-2))), Decimal(200)) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala new file mode 100644 index 000000000000..c734a9cfbbdc --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenMiscExpressionsSuite extends MiscExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala new file mode 100644 index 000000000000..34830b368cae --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNondeterministicSuite extends NondeterministicSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala new file mode 100644 index 000000000000..900fd764d0d9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNullExpressionsSuite extends NullExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala new file mode 100644 index 000000000000..90e93f3593ee --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenPredicateSuite extends PredicateSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala new file mode 100644 index 000000000000..95d2e71ffe59 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRandomSuite extends RandomSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala new file mode 100644 index 000000000000..33cb9a783585 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRegexpExpressionsSuite extends RegexpExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala new file mode 100644 index 000000000000..37c630f495f2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenSortOrderExpressionsSuite extends SortOrderExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala new file mode 100644 index 000000000000..cdb67efeccf3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenStringExpressionsSuite extends StringExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala new file mode 100644 index 000000000000..063b64698113 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala @@ -0,0 +1,217 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, ALL_TIMEZONES, UTC, UTC_OPT} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{fromJavaTimestamp, millisToMicros, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BinaryType, ByteType, DateType, Decimal, DecimalType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, TimestampType} +import org.apache.spark.util.DebuggableThreadUtils + +import java.sql.{Date, Timestamp} +import java.util.{Calendar, TimeZone} + +class GlutenTryCastSuite extends TryCastSuite with GlutenTestsTrait { + + testGluten("data type casting") { + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts)) + + // SystemV timezones are a legacy way of specifying timezones in Unix-like OS. + // It is not supported by Velox. + for (tz <- ALL_TIMEZONES.filterNot(_.getId.contains("SystemV"))) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz.getId + ) { + val timeZoneId = Option(tz.getId) + var c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 2, 8, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 10, 1, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + } + } + + checkEvaluation(cast("abdef", StringType), "abdef") + checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) + + checkEvaluation(cast(cast(sd, DateType), StringType), sd) + checkEvaluation(cast(cast(d, StringType), DateType), 0) + + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get + ) { + checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) + checkEvaluation( + cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), + fromJavaTimestamp(ts)) + + // all convert to string type to check + checkEvaluation( + cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType), + sd) + checkEvaluation( + cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT), + zts) + } + + checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") + + checkEvaluation( + cast( + cast(cast(cast(cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), + LongType), + 5.toLong) + + checkEvaluation(cast("23", DoubleType), 23d) + checkEvaluation(cast("23", IntegerType), 23) + checkEvaluation(cast("23", FloatType), 23f) + checkEvaluation(cast("23", DecimalType.USER_DEFAULT), Decimal(23)) + checkEvaluation(cast("23", ByteType), 23.toByte) + checkEvaluation(cast("23", ShortType), 23.toShort) + checkEvaluation(cast(123, IntegerType), 123) + + checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) + } + + testGluten("cast string to timestamp") { + DebuggableThreadUtils.parmap( + ALL_TIMEZONES + .filterNot(_.getId.contains("SystemV")) + .filterNot(_.getId.contains("Europe/Kyiv")) + .filterNot(_.getId.contains("America/Ciudad_Juarez")) + .filterNot(_.getId.contains("Antarctica/Vostok")) + .filterNot(_.getId.contains("Pacific/Kanton")) + .filterNot(_.getId.contains("Asia/Tehran")) + .filterNot(_.getId.contains("Iran")), + prefix = "CastSuiteBase-cast-string-to-timestamp", + maxThreads = 1 + ) { + zid => + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { + checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) + } + + val tz = TimeZone.getTimeZone(zid) + var c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 ", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18 12:03:17", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17+7:3", + // new Timestamp(c.getTimeInMillis)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18 12:03:17.123", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123-1:0", + // new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123-01:00", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123+07:30", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123+7:3", + // new Timestamp(c.getTimeInMillis)) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala new file mode 100644 index 000000000000..6af97677e5d8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenTryEvalSuite extends TryEvalSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala new file mode 100644 index 000000000000..5f89c2810e6e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenPercentileSuite extends PercentileSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala new file mode 100644 index 000000000000..4099ea138227 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSessionCatalogSuite + extends DataSourceV2DataFrameSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala new file mode 100644 index 000000000000..327c930bfb3f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSuite + extends DataSourceV2DataFrameSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala new file mode 100644 index 000000000000..10f4d90f54f5 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2FunctionSuite + extends DataSourceV2FunctionSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala new file mode 100644 index 000000000000..7e1a1cdaca9a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2SQLSessionCatalogSuite + extends DataSourceV2SQLSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala new file mode 100644 index 000000000000..ff7618008680 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql._ + +class GlutenDataSourceV2SQLSuiteV1Filter + extends DataSourceV2SQLSuiteV1Filter + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala new file mode 100644 index 000000000000..7e02fc07cec0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql._ + +class GlutenDataSourceV2SQLSuiteV2Filter + extends DataSourceV2SQLSuiteV2Filter + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala new file mode 100644 index 000000000000..803c407ca8e7 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala @@ -0,0 +1,167 @@ +/* + * 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.spark.sql.connector + +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions.row_number +import org.apache.spark.sql.internal.SQLConf + +import test.org.apache.spark.sql.connector.{JavaOrderAndPartitionAwareDataSource, JavaPartitionAwareDataSource} + +class GlutenDataSourceV2Suite extends DataSourceV2Suite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten("partitioning reporting") { + import org.apache.spark.sql.functions.{count, sum} + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + Seq(classOf[PartitionAwareDataSource], classOf[JavaPartitionAwareDataSource]).foreach { + cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, Seq(Row(1, 4), Row(1, 4), Row(3, 6), Row(2, 6), Row(4, 2), Row(4, 2))) + + val groupByColA = df.groupBy('i).agg(sum('j)) + checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4))) + assert(collectFirst(groupByColA.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColAB = df.groupBy('i, 'j).agg(count("*")) + checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2))) + assert(collectFirst(groupByColAB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColB = df.groupBy('j).agg(sum('i)) + checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) + assert(collectFirst(groupByColB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + + val groupByAPlusB = df.groupBy('i + 'j).agg(count("*")) + checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) + assert(collectFirst(groupByAPlusB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + } + } + } + } + + testGluten("ordering and partitioning reporting") { + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + Seq( + classOf[OrderAndPartitionAwareDataSource], + classOf[JavaOrderAndPartitionAwareDataSource] + ).foreach { + cls => + withClue(cls.getName) { + // we test report ordering (together with report partitioning) with these + // transformations: + // - groupBy("i").flatMapGroups: + // hash-partitions by "i" and sorts each partition by "i" + // requires partitioning and sort by "i" + // - aggregation function over window partitioned by "i" and ordered by "j": + // hash-partitions by "i" and sorts each partition by "j" + // requires partitioning by "i" and sort by "i" and "j" + Seq( + // with no partitioning and no order, we expect shuffling AND sorting + (None, None, (true, true), (true, true)), + // partitioned by i and no order, we expect NO shuffling BUT sorting + (Some("i"), None, (false, true), (false, true)), + // partitioned by i and in-partition sorted by i, + // we expect NO shuffling AND sorting for groupBy but sorting for window function + (Some("i"), Some("i"), (false, false), (false, true)), + // partitioned by i and in-partition sorted by j, we expect NO shuffling BUT sorting + (Some("i"), Some("j"), (false, true), (false, true)), + // partitioned by i and in-partition sorted by i,j, we expect NO shuffling NOR sorting + (Some("i"), Some("i,j"), (false, false), (false, false)), + // partitioned by j and in-partition sorted by i, we expect shuffling AND sorting + (Some("j"), Some("i"), (true, true), (true, true)), + // partitioned by j and in-partition sorted by i,j, we expect shuffling and sorting + (Some("j"), Some("i,j"), (true, true), (true, true)) + ).foreach { + testParams => + val (partitionKeys, orderKeys, groupByExpects, windowFuncExpects) = testParams + + withClue(f"${partitionKeys.orNull} ${orderKeys.orNull}") { + val df = spark.read + .option("partitionKeys", partitionKeys.orNull) + .option("orderKeys", orderKeys.orNull) + .format(cls.getName) + .load() + checkAnswer( + df, + Seq(Row(1, 4), Row(1, 5), Row(3, 5), Row(2, 6), Row(4, 1), Row(4, 2))) + + // groupBy(i).flatMapGroups + { + val groupBy = df + .groupBy($"i") + .as[Int, (Int, Int)] + .flatMapGroups { + (i: Int, it: Iterator[(Int, Int)]) => Iterator.single((i, it.length)) + } + checkAnswer( + groupBy.toDF(), + Seq(Row(1, 2), Row(2, 1), Row(3, 1), Row(4, 2)) + ) + + val (shuffleExpected, sortExpected) = groupByExpects + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined === shuffleExpected) + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: SortExecTransformer => e + }.isDefined === sortExpected) + } + + // aggregation function over window partitioned by i and ordered by j + { + val windowPartByColIOrderByColJ = df.withColumn( + "no", + row_number().over(Window.partitionBy(Symbol("i")).orderBy(Symbol("j")))) + checkAnswer( + windowPartByColIOrderByColJ, + Seq( + Row(1, 4, 1), + Row(1, 5, 2), + Row(2, 6, 1), + Row(3, 5, 1), + Row(4, 1, 1), + Row(4, 2, 2) + )) + + val (shuffleExpected, sortExpected) = windowFuncExpects + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined === shuffleExpected) + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: SortExecTransformer => e + }.isDefined === sortExpected) + } + } + } + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala new file mode 100644 index 000000000000..ea2fc4e943e1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeleteFromTableSuite + extends GroupBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala new file mode 100644 index 000000000000..74893c5e51a2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedDeleteFromTableSuite + extends DeltaBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala new file mode 100644 index 000000000000..2ca5d06f9907 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedMergeIntoTableSuite + extends DeltaBasedMergeIntoTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala new file mode 100644 index 000000000000..47a3670d065f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite + extends DeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala new file mode 100644 index 000000000000..dd4f93140c48 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite + extends DeltaBasedUpdateAsDeleteAndInsertTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala new file mode 100644 index 000000000000..b173c743a99f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedUpdateTableSuite + extends DeltaBasedUpdateTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala new file mode 100644 index 000000000000..02198099aa4b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.spark.sql.connector + +import org.apache.gluten.execution.FileSourceScanExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.QueryExecutionListener + +import scala.collection.Seq +import scala.collection.mutable.ArrayBuffer + +class GlutenFileDataSourceV2FallBackSuite + extends FileDataSourceV2FallBackSuite + with GlutenSQLTestsBaseTrait { + + testGluten("Fallback Parquet V2 to V1") { + Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { + format => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] + val exceptions = ArrayBuffer.empty[(String, Exception)] + val listener = new QueryExecutionListener { + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = { + exceptions += funcName -> exception + } + + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + commands += funcName -> qe.logical + } + } + spark.listenerManager.register(listener) + + try { + withTempPath { + path => + val inputData = spark.range(10) + inputData.write.format(format).save(path.getCanonicalPath) + sparkContext.listenerBus.waitUntilEmpty() + assert(commands.length == 1) + assert(commands.head._1 == "command") + assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) + assert( + commands.head._2 + .asInstanceOf[InsertIntoHadoopFsRelationCommand] + .fileFormat + .isInstanceOf[ParquetFileFormat]) + val df = spark.read.format(format).load(path.getCanonicalPath) + checkAnswer(df, inputData.toDF()) + assert( + df.queryExecution.executedPlan.exists( + _.isInstanceOf[FileSourceScanExecTransformer])) + } + } finally { + spark.listenerManager.unregister(listener) + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala new file mode 100644 index 000000000000..25f377505c69 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenGroupBasedDeleteFromTableSuite + extends GroupBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala new file mode 100644 index 000000000000..9bf7abb2b70f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenGroupBasedMergeIntoTableSuite + extends GroupBasedMergeIntoTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala new file mode 100644 index 000000000000..ef87b504002e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala @@ -0,0 +1,975 @@ +/* + * 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.spark.sql.connector + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.SortMergeJoinExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.distributions.Distributions +import org.apache.spark.sql.connector.expressions.Expressions.{bucket, days, identity} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.{ColumnarShuffleExchangeExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import java.util.Collections + +class GlutenKeyGroupedPartitioningSuite + extends KeyGroupedPartitioningSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set("spark.sql.adaptive.enabled", "false") + .set("spark.sql.shuffle.partitions", "5") + } + + private val emptyProps: java.util.Map[String, String] = { + Collections.emptyMap[String, String] + } + private def createTable( + table: String, + columns: Array[Column], + partitions: Array[Transform], + catalog: InMemoryTableCatalog = catalog): Unit = { + catalog.createTable( + Identifier.of(Array("ns"), table), + columns, + partitions, + emptyProps, + Distributions.unspecified(), + Array.empty, + None, + None, + numRowsPerSplit = 1) + } + + private def collectColumnarShuffleExchangeExec( + plan: SparkPlan): Seq[ColumnarShuffleExchangeExec] = { + // here we skip collecting shuffle operators that are not associated with SMJ + collect(plan) { + case s: SortMergeJoinExecTransformer => s + case s: SortMergeJoinExec => s + }.flatMap(smj => collect(smj) { case s: ColumnarShuffleExchangeExec => s }) + } + private def collectScans(plan: SparkPlan): Seq[BatchScanExec] = { + collect(plan) { case s: BatchScanExec => s } + } + + private val customers: String = "customers" + private val customersColumns: Array[Column] = Array( + Column.create("customer_name", StringType), + Column.create("customer_age", IntegerType), + Column.create("customer_id", LongType)) + + private val orders: String = "orders" + private val ordersColumns: Array[Column] = + Array(Column.create("order_amount", DoubleType), Column.create("customer_id", LongType)) + + private def testWithCustomersAndOrders( + customers_partitions: Array[Transform], + orders_partitions: Array[Transform], + expectedNumOfShuffleExecs: Int): Unit = { + createTable(customers, customersColumns, customers_partitions) + sql( + s"INSERT INTO testcat.ns.$customers VALUES " + + s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)") + + createTable(orders, ordersColumns, orders_partitions) + sql( + s"INSERT INTO testcat.ns.$orders VALUES " + + s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)") + + val df = sql( + "SELECT customer_name, customer_age, order_amount " + + s"FROM testcat.ns.$customers c JOIN testcat.ns.$orders o " + + "ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.length == expectedNumOfShuffleExecs) + + checkAnswer( + df, + Seq( + Row("aaa", 10, 100.0), + Row("aaa", 10, 200.0), + Row("bbb", 20, 150.0), + Row("bbb", 20, 250.0), + Row("bbb", 20, 350.0), + Row("ccc", 30, 400.50))) + } + + testGluten("partitioned join: only one side reports partitioning") { + val customers_partitions = Array(bucket(4, "customer_id")) + val orders_partitions = Array(bucket(2, "customer_id")) + + testWithCustomersAndOrders(customers_partitions, orders_partitions, 2) + } + testGluten("partitioned join: exact distribution (same number of buckets) from both sides") { + val customers_partitions = Array(bucket(4, "customer_id")) + val orders_partitions = Array(bucket(4, "customer_id")) + + testWithCustomersAndOrders(customers_partitions, orders_partitions, 0) + } + + private val items: String = "items" + private val itemsColumns: Array[Column] = Array( + Column.create("id", LongType), + Column.create("name", StringType), + Column.create("price", FloatType), + Column.create("arrive_time", TimestampType)) + private val purchases: String = "purchases" + private val purchasesColumns: Array[Column] = Array( + Column.create("item_id", LongType), + Column.create("price", FloatType), + Column.create("time", TimestampType)) + + testGluten( + "SPARK-41413: partitioned join: partition values" + + " from one side are subset of those from the other side") { + val items_partitions = Array(bucket(4, "id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(4, "item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(3, "bb", 10.0, 19.5))) + } + } + } + + testGluten("SPARK-41413: partitioned join: partition values from both sides overlaps") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(2, 19.5, cast('2020-02-01' as timestamp)), " + + "(4, 30.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(2, "bb", 10.0, 19.5))) + } + } + } + + testGluten("SPARK-41413: partitioned join: non-overlapping partition values from both sides") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(4, 42.0, cast('2020-01-01' as timestamp)), " + + "(5, 19.5, cast('2020-02-01' as timestamp)), " + + "(6, 30.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq.empty) + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with same partition keys and one side fully clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 3)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not contain any shuffle") + if (pushDownValues) { + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with same partition keys and both sides partially clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(1, 55.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 22.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 7), ("false", 3)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not contain any shuffle") + if (pushDownValues) { + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 40.0, 55.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(1, "aa", 41.0, 55.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 10.0, 22.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered: with different" + + " partition keys and both sides partially clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(1, 55.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 25.0, cast('2020-01-03' as timestamp)), " + + s"(2, 30.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 10), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 40.0, 55.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(1, "aa", 41.0, 55.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 10.0, 25.0), + Row(2, "bb", 10.0, 30.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered: with different" + + " partition keys and missing keys on left-hand side") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 25.0, cast('2020-01-03' as timestamp)), " + + s"(2, 30.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 9), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with different partition keys and missing keys on right-hand side") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 6), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq(Row(2, "bb", 10.0, 15.0), Row(2, "bb", 10.0, 20.0), Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: left outer join") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 15.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 20.0, cast('2020-01-01' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + // In a left-outer join, and when the left side has larger stats, partially clustered + // distribution should kick in and pick the right hand side to replicate partitions. + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 7), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i LEFT JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, null), + Row(1, "aa", 41.0, null), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 15.0, null), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: right outer join") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + + s"(2, 20.0, cast('2020-01-01' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + // The left-hand side is picked as the side to replicate partitions based on stats, but since + // this is right outer join, partially clustered distribution won't kick in, and Spark should + // only push down partition values on both side. + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i RIGHT JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.map(_.inputRDD.partitions.length).toSet.size == 1) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(null, null, null, 25.0), + Row(null, null, null, 30.0), + Row(1, "aa", 40.0, 45.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: full outer join is not applicable") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + + s"(2, 20.0, cast('2020-01-02' as timestamp)), " + + s"(3, 20.0, cast('2020-01-01' as timestamp)), " + + s"(4, 25.0, cast('2020-01-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i FULL OUTER JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.map(_.inputRDD.partitions.length).toSet.size == 1) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(null, null, null, 20.0), + Row(null, null, null, 25.0), + Row(null, null, null, 30.0), + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 41.0, null), + Row(2, "bb", 10.0, 15.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten("SPARK-44641: duplicated records when SPJ is not triggered") { + val items_partitions = Array(bucket(8, "id")) + createTable(items, itemsColumns, items_partitions) + sql(s""" + INSERT INTO testcat.ns.$items VALUES + (1, 'aa', 40.0, cast('2020-01-01' as timestamp)), + (1, 'aa', 41.0, cast('2020-01-15' as timestamp)), + (2, 'bb', 10.0, cast('2020-01-01' as timestamp)), + (2, 'bb', 10.5, cast('2020-01-01' as timestamp)), + (3, 'cc', 15.5, cast('2020-02-01' as timestamp))""") + + val purchases_partitions = Array(bucket(8, "item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql(s"""INSERT INTO testcat.ns.$purchases VALUES + (1, 42.0, cast('2020-01-01' as timestamp)), + (1, 44.0, cast('2020-01-15' as timestamp)), + (1, 45.0, cast('2020-01-15' as timestamp)), + (2, 11.0, cast('2020-01-01' as timestamp)), + (3, 19.5, cast('2020-02-01' as timestamp))""") + + Seq(true, false).foreach { + pushDownValues => + Seq(true, false).foreach { + partiallyClusteredEnabled => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClusteredEnabled.toString + ) { + + // join keys are not the same as the partition keys, therefore SPJ is not triggered. + val df = sql(s""" + SELECT id, name, i.price as purchase_price, p.item_id, p.price as sale_price + FROM testcat.ns.$items i JOIN testcat.ns.$purchases p + ON i.arrive_time = p.time ORDER BY id, purchase_price, p.item_id, sale_price + """) + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.nonEmpty, "shuffle should exist when SPJ is not used") + + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 1, 42.0), + Row(1, "aa", 40.0, 2, 11.0), + Row(1, "aa", 41.0, 1, 44.0), + Row(1, "aa", 41.0, 1, 45.0), + Row(2, "bb", 10.0, 1, 42.0), + Row(2, "bb", 10.0, 2, 11.0), + Row(2, "bb", 10.5, 1, 42.0), + Row(2, "bb", 10.5, 2, 11.0), + Row(3, "cc", 15.5, 3, 19.5) + ) + ) + } + } + } + } + + testGluten("partitioned join: join with two partition keys and matching & sorted partitions") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 41.0, 44.0), + Row(1, "aa", 41.0, 45.0), + Row(2, "bb", 10.0, 11.0), + Row(2, "bb", 10.5, 11.0), + Row(3, "cc", 15.5, 19.5))) + } + + testGluten("partitioned join: join with two partition keys and unsorted partitions") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 41.0, 44.0), + Row(1, "aa", 41.0, 45.0), + Row(2, "bb", 10.0, 11.0), + Row(2, "bb", 10.5, 11.0), + Row(3, "cc", 15.5, 19.5))) + } + + testGluten("partitioned join: join with two partition keys and different # of partition keys") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp))") + + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.nonEmpty, "should add shuffle when partition keys mismatch") + } + + testGluten("data source partitioning + dynamic partition filtering") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10" + ) { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + // number of unique partitions changed after dynamic filtering - the gap + // should be filled with empty partitions and the job should still succeed + var df = sql( + s"SELECT sum(p.price) from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price > 40.0") + checkAnswer(df, Seq(Row(131))) + + // dynamic filtering doesn't change partitioning so storage-partitioned join should kick + // in + df = sql( + s"SELECT sum(p.price) from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price >= 10.0") + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer(df, Seq(Row(303.5))) + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala new file mode 100644 index 000000000000..735b5d1a0e1b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenLocalScanSuite extends LocalScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala new file mode 100644 index 000000000000..59a14fb11c00 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenMetadataColumnSuite extends MetadataColumnSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala new file mode 100644 index 000000000000..92f2a04cebe1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSupportsCatalogOptionsSuite + extends SupportsCatalogOptionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala new file mode 100644 index 000000000000..93502b7adb05 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTableCapabilityCheckSuite + extends TableCapabilityCheckSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala new file mode 100644 index 000000000000..f96ec9a6d1df --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.spark.sql.connector + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenWriteDistributionAndOrderingSuite + extends WriteDistributionAndOrderingSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala new file mode 100644 index 000000000000..6c14c16664a7 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsDSv2Suite + extends QueryCompilationErrorsDSv2Suite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala new file mode 100644 index 000000000000..7ccb3b059ac6 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsSuite + extends QueryCompilationErrorsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala new file mode 100644 index 000000000000..8c661f86023a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.spark.sql.errors + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryExecutionErrorsSuite + extends QueryExecutionErrorsSuite + with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = { + // Disables VeloxAppendBatches in which GeneralOutIterator wraps vanilla Spark's exceptions + // with GlutenException. + super.sparkConf + .set("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput", "false") + } + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala new file mode 100644 index 000000000000..307a740396ea --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryParsingErrorsSuite extends QueryParsingErrorsSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala new file mode 100644 index 000000000000..de2436a39e33 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -0,0 +1,257 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions +import org.apache.gluten.extension.caller.CallerInfo +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.rules.Rule + +class FallbackStrategiesSuite extends GlutenSQLTestsTrait { + import FallbackStrategiesSuite._ + + testGluten("Fall back the whole query if one unsupported") { + withSQLConf((GlutenConfig.COLUMNAR_QUERY_FALLBACK_THRESHOLD.key, "1")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + + testGluten("Fall back the whole plan if meeting the configured threshold") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "1")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + } + + testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "4")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + } + + testGluten( + "Fall back the whole plan if meeting the configured threshold (leaf node is" + + " transformable)") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "2")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + } + + testGluten( + "Don't Fall back the whole plan if NOT meeting the configured threshold (" + + "leaf node is transformable)") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "3")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + } + + testGluten("Tag not transformable more than once") { + val originalPlan = UnaryOp1(LeafOp(supportsColumnar = true)) + FallbackTags.add(originalPlan, "fake reason") + val rule = FallbackEmptySchemaRelation() + val newPlan = rule.apply(originalPlan) + val reason = FallbackTags.get(newPlan).reason() + assert( + reason.contains("fake reason") && + reason.contains("at least one of its children has empty output")) + } + + testGluten("test enabling/disabling Gluten at thread level") { + spark.sql("create table fallback_by_thread_config (a int) using parquet") + spark.sql("insert overwrite fallback_by_thread_config select id as a from range(3)") + val sql = + """ + |select * + |from fallback_by_thread_config as t0 + |""".stripMargin + + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + + val thread = new Thread( + () => { + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") + val fallbackPlan = spark.sql(sql).queryExecution.executedPlan + val fallbackScanExec = fallbackPlan.collect { + case e: FileSourceScanExec if !e.isInstanceOf[BasicScanExecTransformer] => true + } + assert(fallbackScanExec.size == 1) + + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, null) + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + }) + thread.start() + thread.join(10000) + } +} + +private object FallbackStrategiesSuite { + def newRuleApplier( + spark: SparkSession, + transformBuilders: Seq[ColumnarRuleCall => Rule[SparkPlan]]): HeuristicApplier = { + new HeuristicApplier( + spark, + transformBuilders, + List(c => p => ExpandFallbackPolicy(c.caller.isAqe(), p)), + List( + c => RemoveTopmostColumnarToRow(c.session, c.caller.isAqe()), + _ => ColumnarCollapseTransformStages(GlutenConfig.get) + ), + List(_ => RemoveFallbackTagRule()), + Nil + ) + } + + // TODO: Generalize the code among shim versions. + case class FallbackEmptySchemaRelation() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case p => + if (p.children.exists(_.output.isEmpty)) { + // Some backends are not eligible to offload plan with zero-column input. + // If any child have empty output, mark the plan and that child as UNSUPPORTED. + FallbackTags.add(p, "at least one of its children has empty output") + p.children.foreach { + child => + if (child.output.isEmpty) { + FallbackTags.add(child, "at least one of its children has empty output") + } + } + } + p + } + } + + case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = Seq.empty + } + + case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 = + copy(child = newChild) + } + + case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 = + copy(child = newChild) + } + +// For replacing LeafOp. + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = Seq.empty + } + +// For replacing UnaryOp1. + case class UnaryOp1Transformer(override val child: SparkPlan) + extends UnaryExecNode + with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = + copy(child = newChild) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala new file mode 100644 index 000000000000..e689c2a3c1e3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala @@ -0,0 +1,59 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.broadcast.TorrentBroadcast +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestsBaseTrait} +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.broadcast + +class GlutenBroadcastExchangeSuite extends BroadcastExchangeSuite with GlutenSQLTestsBaseTrait {} + +// Additional tests run in 'local-cluster' mode. +class GlutenLocalBroadcastExchangeSuite + extends SparkFunSuite + with LocalSparkContext + with GlutenTestsBaseTrait + with AdaptiveSparkPlanHelper { + + def newSparkConf(): SparkConf = { + val conf = new SparkConf().setMaster("local-cluster[2,1,1024]") + GlutenSQLTestsBaseTrait.nativeSparkConf(conf, warehouse) + } + + test("SPARK-39983 - Broadcasted relation is not cached on the driver") { + // Use distributed cluster as in local mode the broabcast value is actually cached. + val conf = newSparkConf() + sc = new SparkContext(conf) + val spark = new SparkSession(sc) + + val df = spark.range(1).toDF() + val joinDF = df.join(broadcast(df), "id") + joinDF.collect() + val broadcastExchangeExec = collect(joinDF.queryExecution.executedPlan) { + case p: ColumnarBroadcastExchangeExec => p + } + assert(broadcastExchangeExec.size == 1, "one and only ColumnarBroadcastExchangeExec") + + // The broadcasted relation should not be cached on the driver. + val broadcasted = + broadcastExchangeExec(0).relationFuture.get().asInstanceOf[TorrentBroadcast[Any]] + assert(!broadcasted.hasCachedValue) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala new file mode 100644 index 000000000000..8e5c71735aca --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala @@ -0,0 +1,292 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, SparkSession} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +class GlutenCoalesceShufflePartitionsSuite + extends CoalesceShufflePartitionsSuite + with GlutenTestsCommonTrait { + + override protected def afterAll(): Unit = {} + + override def withSparkSession( + f: SparkSession => Unit, + targetPostShuffleInputSize: Int, + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { + val sparkConf = + new SparkConf(false) + .setMaster("local[*]") + .setAppName("test") + .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") + .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") + .set(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, targetPostShuffleInputSize.toString) + .set(SQLConf.COALESCE_PARTITIONS_ENABLED.key, "true") + // Gluten config + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "5g") + minNumPostShufflePartitions match { + case Some(numPartitions) => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, numPartitions.toString) + case None => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, "1") + } + + val spark = SparkSession + .builder() + .config(sparkConf) + .getOrCreate() + try f(spark) + finally { + spark.stop() + } + } + + Seq(Some(5), None).foreach { + minNumPostShufflePartitions => + val testNameNote = minNumPostShufflePartitions match { + case Some(numPartitions) => "(minNumPostShufflePartitions: " + numPartitions + ")" + case None => "" + } + + // Ported from vanilla spark with targetPostShuffleInputSize changed. + testGluten(s"determining the number of reducers: aggregate operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 20 as key", "id as value") + val agg = df.groupBy("key").count() + + // Check the answer first. + QueryTest.checkAnswer(agg, spark.range(0, 20).selectExpr("id", "50 as cnt").collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = agg.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + case None => + assert(shuffleReads.length === 1) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + // Change the original value 2000 to 2500 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 2500, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: join operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "id as value") + .union(spark.range(0, 1000).selectExpr("id % 500 as key", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + // Change the original value 16384 to 20000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 20000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: complex query 1$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + .groupBy("key2") + .count() + .toDF("key2", "cnt2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("cnt2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 500) + .selectExpr("id", "2 as cnt") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + + // Change the original value 16384 to 20000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 20000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: complex query 2$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = + df1 + .join(df2, col("key1") === col("key2")) + .select(col("key1"), col("cnt1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "2 as cnt", "id as value") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + + // Change the original value 12000 to 16000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 16000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: plan already partitioned$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + try { + spark.range(1000).write.bucketBy(30, "id").saveAsTable("t") + // `df1` is hash partitioned by `id`. + val df1 = spark.read.table("t") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("id") === col("key2")).select(col("id"), col("value2")) + + // Check the answer first. + val expectedAnswer = spark + .range(0, 500) + .selectExpr("id % 500", "id as value") + .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's make sure we do not reduce number of post shuffle partitions. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + assert(shuffleReads.length === 0) + } finally { + spark.sql("drop table t") + } + } + withSparkSession(test, 12000, minNumPostShufflePartitions) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala new file mode 100644 index 000000000000..bc15153cca09 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala @@ -0,0 +1,72 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} +import org.apache.spark.sql.internal.SQLConf + +class GlutenExchangeSuite extends ExchangeSuite with GlutenSQLTestsBaseTrait { + + testGluten("Exchange reuse across the whole plan with shuffle partition 2") { + // The shuffle exchange will be inserted between Aggregate + // when shuffle partition is > 1. + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "2") { + val df = sql(""" + |SELECT + | (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key), + | a.key + |FROM testData AS a + |JOIN testData AS b ON b.key = a.key + """.stripMargin) + + val plan = df.queryExecution.executedPlan + + val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id } + val reusedExchangeIds = plan.collectWithSubqueries { + case re: ReusedExchangeExec => re.child.id + } + + assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly") + assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly") + assert( + reusedExchangeIds.forall(exchangeIds.contains(_)), + "ReusedExchangeExec should reuse an existing exchange") + + val df2 = sql(""" + |SELECT + | (SELECT min(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key), + | (SELECT max(a.key) FROM testData AS a JOIN testData2 AS b ON b.a = a.key) + """.stripMargin) + + val plan2 = df2.queryExecution.executedPlan + + val exchangeIds2 = plan2.collectWithSubqueries { case e: Exchange => e.id } + val reusedExchangeIds2 = plan2.collectWithSubqueries { + case re: ReusedExchangeExec => re.child.id + } + + assert(exchangeIds2.size == 4, "Whole plan exchange reusing not working correctly") + assert(reusedExchangeIds2.size == 2, "Whole plan exchange reusing not working correctly") + assert( + reusedExchangeIds2.forall(exchangeIds2.contains(_)), + "ReusedExchangeExec should reuse an existing exchange") + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala new file mode 100644 index 000000000000..281958455855 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala @@ -0,0 +1,107 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils + +import org.apache.logging.log4j.{Level, LogManager} +import org.apache.logging.log4j.core.LoggerContext + +import scala.io.Source + +class GlutenQueryExecutionSuite extends QueryExecutionSuite with GlutenSQLTestsBaseTrait { + + override def checkDumpedPlans(path: String, expected: Int): Unit = + Utils.tryWithResource(Source.fromFile(path)) { + source => + assert( + source.getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") + .map(_.replaceAll("#\\d+", "#x")) == List( + "== Parsed Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Analyzed Logical Plan ==", + "id: bigint", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Optimized Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Physical Plan ==", + "*(1) ColumnarToRow", + s"+- ColumnarRange 0, $expected, 1, 2, $expected, [id#xL]", + "" + )) + } + + testGluten("dumping query execution info to a file - explainMode=formatted") { + withTempDir { + dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path, explainMode = Option("formatted")) + val lines = Utils.tryWithResource(Source.fromFile(path))(_.getLines().toList) + assert( + lines + .takeWhile(_ != "== Whole Stage Codegen ==") + .map(_.replaceAll("#\\d+", "#x")) == List( + "== Physical Plan ==", + "* ColumnarToRow (2)", + "+- ColumnarRange (1)", + "", + "", + "(1) ColumnarRange", + "Output [1]: [id#xL]", + "Arguments: 0, 10, 1, 2, 10, [id#xL]", + "", + "(2) ColumnarToRow [codegen id : 1]", + "Input [1]: [id#xL]", + "", + "" + )) + } + } + + testGluten("Logging plan changes for execution") { + val ctx = LogManager.getContext(false).asInstanceOf[LoggerContext] + val config = ctx.getConfiguration + val loggerConfig = config.getLoggerConfig(LogManager.ROOT_LOGGER_NAME) + loggerConfig.setLevel(Level.INFO) + ctx.updateLoggers() + + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "INFO" + ) { + spark.range(1).groupBy("id").count().queryExecution.executedPlan + } + } + Seq("=== Applying Rule org.apache.spark.sql.execution", "=== Result of Batch Preparations ===") + .foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg) + ) + ) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala new file mode 100644 index 000000000000..9d819d2bd90f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenRemoveRedundantWindowGroupLimitsSuite + extends RemoveRedundantWindowGroupLimitsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala new file mode 100644 index 000000000000..33bf1a1ec97e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -0,0 +1,143 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf + +class GlutenReplaceHashWithSortAggSuite + extends ReplaceHashWithSortAggSuite + with GlutenSQLTestsBaseTrait { + + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { + case s @ (_: HashAggregateExecBaseTransformer | _: ObjectHashAggregateExec) => s + }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } + + private def checkAggs( + query: String, + enabledHashAggCount: Int, + enabledSortAggCount: Int, + disabledHashAggCount: Int, + disabledSortAggCount: Int): Unit = { + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "true") { + val df = sql(query) + checkNumAggs(df, enabledHashAggCount, enabledSortAggCount) + val result = df.collect() + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { + val df = sql(query) + checkNumAggs(df, disabledHashAggCount, disabledSortAggCount) + checkAnswer(df, result) + } + } + } + + // === Following cases override super class's cases === + + testGluten("replace partial hash aggregate with sort aggregate") { + withTempView("t") { + spark.range(100).selectExpr("id as key").repartition(10).createOrReplaceTempView("t") + + Seq("FIRST", "COLLECT_LIST").foreach { + aggExpr => + // Because repartition modification causing the result sort order not same and the + // result not same, so we add order by key before comparing the result. + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT key + | FROM t + | WHERE key > 10 + | SORT BY key + |) + |GROUP BY key + |ORDER BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + testGluten("replace partial and final hash aggregate together with sort aggregate") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { + aggExprInfo => + val query = + s""" + |SELECT key, ${aggExprInfo._1}(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) + } + } + } + + testGluten("do not replace hash aggregate if child does not have sort order") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT /*+ BROADCAST(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + testGluten("do not replace hash aggregate if there is no group-by column") { + withTempView("t1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT $aggExpr(key) + |FROM t1 + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala new file mode 100644 index 000000000000..d7232f6a06c8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenReuseExchangeAndSubquerySuite + extends ReuseExchangeAndSubquerySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala new file mode 100644 index 000000000000..b939c65f34bb --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala @@ -0,0 +1,64 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.internal.SQLConf + +class GlutenSQLAggregateFunctionSuite extends GlutenSQLTestsTrait { + + // TODO: fix in Spark-4.0 + ignoreGluten("GLUTEN-4853: The result order is reversed for count and count distinct") { + val query = + """ + |select count(distinct if(sex = 'x', id, null)) as uv, count(if(sex = 'x', id, null)) as pv + |from values (1, 'x'), (1, 'x'), (2, 'y'), (3, 'x'), (3, 'x'), (4, 'y'), (5, 'x') + |AS tab(id, sex) + |""".stripMargin + val df = sql(query) + checkAnswer(df, Seq(Row(3, 5))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 4) + } + + // TODO: fix in Spark-4.0 + ignoreGluten("Return NaN or null when dividing by zero") { + val query = + """ + |select skewness(value), kurtosis(value) + |from values (1), (1) + |AS tab(value) + |""".stripMargin + val df = sql(query) + + withSQLConf( + SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true" + ) { + checkAnswer(df, Seq(Row(Double.NaN, Double.NaN))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 2) + } + + withSQLConf( + SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> + SQLConf.LEGACY_STATISTICAL_AGGREGATE.defaultValueString + ) { + checkAnswer(df, Seq(Row(null, null))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 2) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala new file mode 100644 index 000000000000..a18d3024fab8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala @@ -0,0 +1,264 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.execution.ColumnarCollectLimitBaseExec + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsTrait, Row} + +class GlutenSQLCollectLimitExecSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } + + private def assertGlutenOperatorMatch[T: reflect.ClassTag]( + df: DataFrame, + checkMatch: Boolean): Unit = { + val executedPlan = getExecutedPlan(df) + + val operatorFound = executedPlan.exists { + plan => + try { + implicitly[reflect.ClassTag[T]].runtimeClass.isInstance(plan) + } catch { + case _: Throwable => false + } + } + + val assertionCondition = operatorFound == checkMatch + val assertionMessage = + if (checkMatch) { + s"Operator ${implicitly[reflect.ClassTag[T]].runtimeClass.getSimpleName} not found " + + s"in executed plan:\n $executedPlan" + } else { + s"Operator ${implicitly[reflect.ClassTag[T]].runtimeClass.getSimpleName} was found " + + s"in executed plan:\n $executedPlan" + } + + assert(assertionCondition, assertionMessage) + } + + test("ColumnarCollectLimitExec - basic limit test") { + val df = spark.range(0, 1000, 1).toDF("id").limit(5) + val expectedData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - with filter") { + val df = spark + .range(0, 20, 1) + .toDF("id") + .filter("id % 2 == 0") + .limit(5) + val expectedData = Seq(Row(0L), Row(2L), Row(4L), Row(6L), Row(8L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - range with repartition") { + + val df = spark + .range(0, 10, 1) + .toDF("id") + .repartition(3) + .orderBy("id") + .limit(3) + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(df, expectedData) + } + + ignore("ColumnarCollectLimitExec - with distinct values") { + val df = spark + .range(0, 10, 1) + .toDF("id") + .select("id") + .distinct() + .limit(5) + val expectedData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - chained limit") { + val df = spark + .range(0, 10, 1) + .toDF("id") + .limit(8) + .limit(3) + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - limit after union") { + val df1 = spark.range(0, 5).toDF("id") + val df2 = spark.range(5, 10).toDF("id") + val unionDf = df1.union(df2).limit(3) + + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(unionDf, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](unionDf, checkMatch = true) + } + + test("ColumnarCollectLimitExec - offset test") { + val df1 = spark.range(0, 10, 1).toDF("id").limit(5).offset(2) + val expectedData1 = Seq(Row(2L), Row(3L), Row(4L)) + + checkAnswer(df1, expectedData1) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df1, checkMatch = true) + + val df2 = spark.range(0, 20, 1).toDF("id").limit(12).offset(5) + val expectedData2 = Seq(Row(5L), Row(6L), Row(7L), Row(8L), Row(9L), Row(10L), Row(11L)) + + checkAnswer(df2, expectedData2) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df2, checkMatch = true) + + val df3 = spark.range(0, 30, 1).toDF("id").limit(10).offset(3) + val expectedData3 = Seq(Row(3L), Row(4L), Row(5L), Row(6L), Row(7L), Row(8L), Row(9L)) + + checkAnswer(df3, expectedData3) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df3, checkMatch = true) + + val df4 = spark.range(0, 15, 1).toDF("id").limit(8).offset(4) + val expectedData4 = Seq(Row(4L), Row(5L), Row(6L), Row(7L)) + + checkAnswer(df4, expectedData4) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df4, checkMatch = true) + + val df5 = spark.range(0, 50, 1).toDF("id").limit(20).offset(10) + val expectedData5 = Seq( + Row(10L), + Row(11L), + Row(12L), + Row(13L), + Row(14L), + Row(15L), + Row(16L), + Row(17L), + Row(18L), + Row(19L)) + + checkAnswer(df5, expectedData5) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df5, checkMatch = true) + } + + test("ColumnarCollectLimitExec - pure offset test") { + val df1 = spark.range(0, 20, 1).toDF("id").offset(5) + val expectedData1 = Seq( + Row(5L), + Row(6L), + Row(7L), + Row(8L), + Row(9L), + Row(10L), + Row(11L), + Row(12L), + Row(13L), + Row(14L), + Row(15L), + Row(16L), + Row(17L), + Row(18L), + Row(19L)) + + checkAnswer(df1, expectedData1) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df1, checkMatch = true) + + val df2 = spark.range(0, 50, 1).toDF("id").offset(10) + val expectedData2 = (10L to 49L).map(Row(_)) + + checkAnswer(df2, expectedData2) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df2, checkMatch = true) + + val df3 = spark.range(0, 100, 2).toDF("id").offset(15) + val expectedData3 = (30L to 98L by 2).map(Row(_)) + + checkAnswer(df3, expectedData3) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df3, checkMatch = true) + + val df4 = spark.range(0, 30, 1).toDF("id").offset(20) + val expectedData4 = (20L to 29L).map(Row(_)) + + checkAnswer(df4, expectedData4) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df4, checkMatch = true) + + val df5 = spark.range(0, 200, 5).toDF("id").offset(10) + val expectedData5 = (50L to 195L by 5).map(Row(_)) + + checkAnswer(df5, expectedData5) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df5, checkMatch = true) + + val df6 = spark.range(0, 5, 1).toDF("id").limit(10) + val expectedData6 = (0L to 4L).map(Row(_)) + + checkAnswer(df6, expectedData6) + } + + test("ColumnarCollectLimitExec - offset with filter") { + val df = spark.range(0, 10, 1).toDF("id").filter("id % 2 == 0").limit(5).offset(2) + val expectedData = Seq(Row(4L), Row(6L), Row(8L)) + + checkAnswer(df, expectedData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - offset after union") { + val df1 = spark.range(0, 5).toDF("id") + val df2 = spark.range(5, 10).toDF("id") + val unionDf = df1.union(df2).limit(6).offset(3) + + val expectedData = Seq(Row(3L), Row(4L), Row(5L)) + checkAnswer(unionDf, expectedData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](unionDf, checkMatch = true) + } + + test("ColumnarCollectLimitExec - single partition with limit, offset, and limit + offset") { + + val singlePartitionDF = spark.range(0, 10, 1).toDF("id").coalesce(1) + + val limitDF = singlePartitionDF.limit(5) + val expectedLimitData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + checkAnswer(limitDF, expectedLimitData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](limitDF, checkMatch = true) + + val offsetDF = singlePartitionDF.offset(3) + val expectedOffsetData = Seq(Row(3L), Row(4L), Row(5L), Row(6L), Row(7L), Row(8L), Row(9L)) + checkAnswer(offsetDF, expectedOffsetData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](offsetDF, checkMatch = true) + + val limitOffsetDF = singlePartitionDF.limit(5).offset(2) + val expectedLimitOffsetData = Seq(Row(2L), Row(3L), Row(4L)) + checkAnswer(limitOffsetDF, expectedLimitOffsetData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](limitOffsetDF, checkMatch = true) + } + +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala new file mode 100644 index 000000000000..7c803dd78d20 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala @@ -0,0 +1,231 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.execution.{SortExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ + +class GlutenSQLWindowFunctionSuite extends SQLWindowFunctionSuite with GlutenSQLTestsTrait { + + private def decimal(v: BigDecimal): Decimal = Decimal(v, 7, 2) + + val customerSchema = StructType( + List( + StructField("c_custkey", IntegerType), + StructField("c_nationkey", IntegerType), + StructField("c_acctbal", DecimalType(7, 2)) + ) + ) + + val customerData = Seq( + Row(4553, 11, decimal(6388.41)), + Row(4953, 10, decimal(6037.28)), + Row(35403, 5, decimal(6034.70)), + Row(35803, 12, decimal(5284.87)), + Row(60865, 5, decimal(-227.82)), + Row(61065, 13, decimal(7284.77)), + Row(127412, 13, decimal(4621.41)), + Row(148303, 10, decimal(4302.30)), + Row(9954, 5, decimal(7587.25)), + Row(95337, 12, decimal(915.61)) + ) + + // TODO: fix in Spark-4.0 + ignore("Literal in window partition by and sort") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT + | c_custkey, + | c_acctbal, + | row_number() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS row_num + |FROM + | customer + |ORDER BY 1, 2; + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(60865, BigDecimal(-22782L, 2), 3), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowExecTransformer => true + case _ => false + } + ) + } + } + + // TODO: fix in Spark-4.0 + ignore("Filter on row number") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT * from (SELECT + | c_custkey, + | c_acctbal, + | row_number() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS row_num + |FROM + | customer ORDER BY 1, 2) where row_num <=2 + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowGroupLimitExecTransformer => true + case _ => false + } + ) + assert( + getExecutedPlan(df).collect { case s: SortExecTransformer if !s.global => s }.size == 1 + ) + } + } + + testGluten("Filter on rank") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT * from (SELECT + | c_custkey, + | c_acctbal, + | rank() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS rank + |FROM + | customer ORDER BY 1, 2) where rank <=2 + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + !getExecutedPlan(df).exists { + case _: WindowGroupLimitExecTransformer => true + case _ => false + } + ) + } + } + + // TODO: fix in Spark-4.0 + ignore("Expression in WindowExpression") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT + | c_custkey, + | avg(c_acctbal) OVER ( + | PARTITION BY c_nationkey + | ORDER BY c_custkey + | ) + |FROM + | customer + |ORDER BY 1, 2; + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(6388410000L, 6)), + Row(4953, BigDecimal(6037280000L, 6)), + Row(9954, BigDecimal(7587250000L, 6)), + Row(35403, BigDecimal(6810975000L, 6)), + Row(35803, BigDecimal(5284870000L, 6)), + Row(60865, BigDecimal(4464710000L, 6)), + Row(61065, BigDecimal(7284770000L, 6)), + Row(95337, BigDecimal(3100240000L, 6)), + Row(127412, BigDecimal(5953090000L, 6)), + Row(148303, BigDecimal(5169790000L, 6)) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowExecTransformer => true + case _ => false + } + ) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala new file mode 100644 index 000000000000..de9a897ffb09 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSameResultSuite extends SameResultSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala new file mode 100644 index 000000000000..ae92e440c902 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala @@ -0,0 +1,98 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.{catalyst, GlutenQueryTestUtil, GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions.{Length, SortOrder} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.length + +class GlutenSortSuite extends SortSuite with GlutenSQLTestsBaseTrait with AdaptiveSparkPlanHelper { + import testImplicits._ + + protected val resolver: Resolver = conf.resolver + + protected def attr(name: String): UnresolvedAttribute = { + UnresolvedAttribute(name) + } + + protected def resolveAttrs[T <: QueryPlan[T]]( + expr: catalyst.expressions.Expression, + plan: QueryPlan[T]): catalyst.expressions.Expression = { + + expr.transform { + case UnresolvedAttribute(Seq(attrName)) => + plan.output.find(attr => resolver(attr.name, attrName)).get + case UnresolvedAttribute(nameParts) => + val attrName = nameParts.mkString(".") + fail(s"cannot resolve a nested attr: $attrName") + } + } + + // TODO: fix in Spark-4.0 + ignore("post-project outputOrdering check") { + val input = Seq( + ("Hello", 4, 2.0), + ("Hello Bob", 10, 1.0), + ("Hello Bob", 1, 3.0) + ) + + val df = input.toDF("a", "b", "c").orderBy(length($"a").desc, $"b".desc) + GlutenQueryTestUtil.checkAnswer( + df, + Seq( + Row("Hello Bob", 10, 1.0), + Row("Hello Bob", 1, 3.0), + Row("Hello", 4, 2.0) + ) + ) + + val ordering = Seq( + catalyst.expressions.SortOrder( + Length(attr("a")), + catalyst.expressions.Descending, + catalyst.expressions.NullsLast, + Seq.empty + ), + catalyst.expressions.SortOrder( + attr("b"), + catalyst.expressions.Descending, + catalyst.expressions.NullsLast, + Seq.empty + ) + ) + + assert( + getExecutedPlan(df).exists { + case _: SortExecTransformer => true + case _ => false + } + ) + val plan = stripAQEPlan(df.queryExecution.executedPlan) + val actualOrdering = plan.outputOrdering + val expectedOrdering = ordering.map(resolveAttrs(_, plan).asInstanceOf[SortOrder]) + assert(actualOrdering.length == expectedOrdering.length) + actualOrdering.zip(expectedOrdering).foreach { + case (actual, expected) => + assert(actual.semanticEquals(expected), "ordering must match") + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala new file mode 100644 index 000000000000..d09576908f78 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.streaming._ + +class GlutenStreamingQuerySuite extends StreamingQuerySuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala new file mode 100644 index 000000000000..bc231e52adc3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTakeOrderedAndProjectSuite + extends TakeOrderedAndProjectSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala new file mode 100644 index 000000000000..70e6c81c1122 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -0,0 +1,1566 @@ +/* + * 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.spark.sql.execution.adaptive.clickhouse + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ColumnarToCarrierRowExecBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.functions.when +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.logging.log4j.Level + +class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + var finalPlanCnt = 0 + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) { + finalPlanCnt += 1 + } + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + checkAnswer(df, result) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + spark.sparkContext.listenerBus.waitUntilEmpty() + // AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that + // exist out of query stages. + val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1) + assert(finalPlanCnt == expectedFinalPlanCnt) + spark.sparkContext.removeSparkListener(listener) + + val exchanges = adaptivePlan.collect { case e: Exchange => e } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def broadcastHashJoinSize(plan: SparkPlan): Int = { + findTopLevelBroadcastHashJoinTransform(plan).size + findTopLevelBroadcastHashJoin(plan).size + } + + private def findTopLevelBroadcastHashJoinTransform( + plan: SparkPlan): Seq[BroadcastHashJoinExecTransformerBase] = { + collect(plan) { case j: BroadcastHashJoinExecTransformerBase => j } + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { case j: BroadcastHashJoinExec => j } + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { case j: SortMergeJoinExec => j } + } + + private def findTopLevelSortMergeJoinTransform( + plan: SparkPlan): Seq[SortMergeJoinExecTransformer] = { + collect(plan) { case j: SortMergeJoinExecTransformer => j } + } + + private def sortMergeJoinSize(plan: SparkPlan): Int = { + findTopLevelSortMergeJoinTransform(plan).size + findTopLevelSortMergeJoin(plan).size + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { case j: ShuffledHashJoinExec => j } + } + + private def findTopLevelShuffledHashJoinTransform( + plan: SparkPlan): Seq[ShuffledHashJoinExecTransformerBase] = { + collect(plan) { case j: ShuffledHashJoinExecTransformerBase => j } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { case j: BaseJoinExec => j } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { case s: SortExec => s } + } + + private def findTopLevelSortTransform(plan: SparkPlan): Seq[SortExecTransformer] = { + collect(plan) { case s: SortExecTransformer => s } + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { + collectWithSubqueries(plan) { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec, _) => e + } + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } + } + + private def checkNumLocalShuffleReads( + plan: SparkPlan, + numShufflesWithoutLocalRead: Int = 0): Unit = { + val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length + + val numLocalReads = collect(plan) { + case r: AQEShuffleReadExec if r.isLocalRead => r + } + // because columnar local reads cannot execute + numLocalReads.foreach { + r => + val rdd = r.executeColumnar() + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + + testGluten("Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Change broadcast join to merge join") { + withTable("t1", "t2") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + sql("CREATE TABLE t1 USING PARQUET AS SELECT 1 c1") + sql("CREATE TABLE t2 USING PARQUET AS SELECT 1 c1") + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM ( + | SELECT distinct c1 from t1 + | ) tmp1 JOIN ( + | SELECT distinct c1 from t2 + | ) tmp2 ON tmp1.c1 = tmp2.c1 + |""".stripMargin) + assert(broadcastHashJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 0) + assert(findTopLevelSortMergeJoinTransform(adaptivePlan).size == 1) + } + } + } + + testGluten("Reuse the parallelism of coalesced shuffle in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + } + } + + testGluten("Reuse the default parallelism in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + val localShuffleRDD0 = localReads(0) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + val localShuffleRDD1 = localReads(1) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + testGluten("Empty stage coalesced to 1-partition RDD") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + val df1 = spark.range(10).withColumn("a", 'id) + val df2 = spark.range(10).withColumn("b", 'id) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[SortMergeJoinExecTransformer]).isDefined) + } + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformerBase]).isDefined) + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } + assert(coalescedReads.length == 3, s"$plan") + coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) + } + } + } + + testGluten("Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeShuffleWithLocalRead' rule, we can convert all the four + // shuffle read to local shuffle read in the bottom two 'BroadcastHashJoin'. + // For the top level 'BroadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten("multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + assert(sortMergeJoinSize(plan) == 3) + // TODO: vanilla spark has 2 bhj, and 1 smj, but gluten has 3 bhj, + // make sure this will not cause performance regression and why it is bhj + assert(broadcastHashJoinSize(adaptivePlan) == 1) + // Vanilla spark still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 4) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + testGluten("Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // // Even with local shuffle read, the query stage reuse can also work. + // gluten change the smj to bhj, stage is changed, so we cannot find the stage with old + // ReuseExchange from stageCache, then the reuse is removed + // https://github.com/apache/spark/pull/24706/ + // files#diff-ec42cd27662f3f528832c298a60fffa1d341feb04aa1d8c80044b70cbe0ebbfcR224 + // maybe vanilla spark should checkReuse rile again + // val ex = findReusedExchange(adaptivePlan) + // assert(ex.size == 1) + } + } + + testGluten("Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + testGluten("Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + testGluten("Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.child.isInstanceOf[ColumnarBroadcastExchangeExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + // Cost is equal, not test cost is greater, need new test, but other test may contain cost change, + // so it maybe not essential + testGluten("Avoid plan change if cost is greater") {} + + testGluten("Change merge join to broadcast join without local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + assert(sortMergeJoinSize(plan) == 2) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten( + "Avoid changing merge join to broadcast join if too many empty partitions " + + "on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5", + // this config will make some empty partitions + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + // because testData2 in gluten sizeInBytes(from ColumnarShuffleExchangeExec plan stats) + // is 78B sometimes, so change the threshold from 80 to 60 + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "60") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.joinBuildSide == BuildRight) + } + } + } + + testGluten("SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "100", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 3 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(query: String, optimizeSkewJoin: Boolean): Unit = { + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult(query) + val innerSmj = findTopLevelSortMergeJoinTransform(innerAdaptivePlan) + assert(innerSmj.size == 1 && innerSmj.head.isSkewJoin == optimizeSkewJoin) + } + + // OptimizeSkewedJoin check the map status, because the + checkSkewJoin("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2", true) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + checkSkewJoin( + "SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 GROUP BY key1", + false) + } + } + } + + testGluten("SPARK-29544: adaptive skew join with different join types") { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { + joinHint => + def getJoinNode(plan: SparkPlan): Seq[BinaryExecNode] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoinTransform(plan) + } else { + findTopLevelShuffledHashJoinTransform(plan) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "800" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + + def checkSkewJoin( + joins: Seq[BinaryExecNode], + leftSkewNum: Int, + rightSkewNum: Int): Unit = { + assert(joins.size == 1) + joins.head match { + case s: SortMergeJoinExecTransformer => assert(s.isSkewJoin) + case g: ShuffledHashJoinExecTransformerBase => assert(g.isSkewJoin) + case _ => assert(false) + } + assert( + joins.head.left + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == leftSkewNum) + assert( + joins.head.right + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == rightSkewNum) + } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "JOIN skewData2 ON key1 = key2") + val inner = getJoinNode(innerAdaptivePlan) + // checkSkewJoin(inner, 2, 1) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData2) */ * FROM skewData1 " + + "LEFT OUTER JOIN skewData2 ON key1 = key2") + val leftJoin = getJoinNode(leftAdaptivePlan) + // checkSkewJoin(leftJoin, 2, 0) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "RIGHT OUTER JOIN skewData2 ON key1 = key2") + val rightJoin = getJoinNode(rightAdaptivePlan) + // checkSkewJoin(rightJoin, 0, 1) + } + } + } + } + + testGluten("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + + testGluten("metrics of the shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(!read.isLocalRead) + assert(!read.hasSkewedPartition) + assert(read.hasCoalescedPartition) + assert( + read.metrics.keys.toSeq.sorted == Seq( + "numCoalescedPartitions", + "numPartitions", + "partitionDataSize")) + assert(read.metrics("numCoalescedPartitions").value == 1) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + assert(read.metrics("partitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformerBase => j }.head + assert(join.joinBuildSide == BuildLeft) + + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(read.isLocalRead) + assert(read.metrics.keys.toSeq == Seq("numPartitions")) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1000" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + } + } + } + } + + testGluten("SPARK-32717: AQEOptimizer should respect excludedRules configuration") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString, + // This test is a copy of test(SPARK-32573), in order to test the configuration + // `spark.sql.adaptive.optimizer.excludedRules` works as expect. + "spark.gluten.sql.columnar.backend.ch.aqe.propagate.empty.relation" -> "false" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData2 t1 WHERE t1.b NOT IN (SELECT b FROM testData3)") + val bhj = findTopLevelBroadcastHashJoin(plan) + assert(bhj.size == 1) + val join = findTopLevelBaseJoin(adaptivePlan) + // this is different compares to test(SPARK-32573) due to the rule + // `EliminateUnnecessaryJoin` has been excluded. + assert(join.nonEmpty) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + // EmptyRelation case + ignore( + GLUTEN_TEST + "SPARK-35455: Unify empty relation optimization " + + "between normal and AQE optimizer - single join") {} + + testGluten("SPARK-32753: Only copy tags to node with no tags") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("v1") { + spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1") + + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { case s: ColumnarShuffleExchangeExec => s }.length == 1) + } + } + } + + testGluten("Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + // this test default level is WARN, so we should check warn level + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "WARN", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80" + ) { + sql( + "SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq( + "=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===" + ).foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg))) + } + } + } + + testGluten("SPARK-33551: Do not use AQE shuffle read for repartition") { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => + s.shuffleOrigin == REPARTITION_BY_COL || s.shuffleOrigin == REPARTITION_BY_NUM + case _ => false + }.isDefined + } + + def checkBHJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + probeSideLocalRead: Boolean, + probeSideCoalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + // There should be only one shuffle that can't do local read, which is either the top shuffle + // from repartition, or BHJ probe side shuffle. + checkNumLocalShuffleReads(plan, 1) + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val bhj = findTopLevelBroadcastHashJoinTransform(plan) + assert(bhj.length == 1) + + // Build side should do local read. + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) + assert(buildSide.isDefined) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) + if (probeSideLocalRead || probeSideCoalescedRead) { + assert(probeSide.isDefined) + if (probeSideLocalRead) { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + } else { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) + } + } else { + assert(probeSide.isEmpty) + } + } + + def checkSMJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + optimizeSkewJoin: Boolean, + coalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.length == 1) + assert(smj.head.isSkewJoin == optimizeSkewJoin) + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } + if (coalescedRead || optimizeSkewJoin) { + assert(aqeReads.length == 2) + if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) + } else { + assert(aqeReads.isEmpty) + } + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val df = sql(""" + |SELECT * FROM ( + | SELECT * FROM testData WHERE key = 1 + |) + |RIGHT OUTER JOIN testData2 + |ON value = b + """.stripMargin) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + // Repartition with no partition num specified. + checkBHJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkBHJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out + // The final plan must have 5 partitions, no optimization can be made to the probe side. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkBHJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + + // Repartition by col and project away the partition cols + checkBHJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + } + + // Force skew join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_ENABLED.key -> "true", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR.key -> "0", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10" + ) { + // Repartition with no partition num specified. + checkSMJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = true) + + // Repartition with default partition num (5 in test env) specified. + checkSMJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out. + // The final plan must have 5 partitions, can't do coalesced read. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkSMJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false) + + // Repartition by col and project away the partition cols + checkSMJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + } + } + } + + testGluten("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "true") { + withTable("t1") { + spark.range(100).selectExpr("id + 1 as a").write.format("parquet").saveAsTable("t1") + val query = "SELECT SUM(a) FROM t1 GROUP BY a" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val metricName = SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED + val blocksFetchedMetric = collectFirst(adaptivePlan) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric.isDefined) + val blocksFetched = blocksFetchedMetric.get.value + withSQLConf(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "false") { + val (_, adaptivePlan2) = runAdaptiveAndVerifyResult(query) + val blocksFetchedMetric2 = collectFirst(adaptivePlan2) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric2.isDefined) + val blocksFetched2 = blocksFetchedMetric2.get.value + assert(blocksFetched == blocksFetched2) + } + } + } + } + + testGluten("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") { + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan + assert(collect(plan) { + case s: ColumnarShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => + s + }.size == 1) + checkAnswer(ds, testData) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + // Pick a small value so that no coalesce can happen. + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2" + ) { + val df = + spark.sparkContext.parallelize((1 to 100).map(i => TestData(i, i.toString)), 10).toDF() + + // partition size [1420, 1420] + checkNoCoalescePartitions(df.repartition($"key"), REPARTITION_BY_COL) + // partition size [1140, 1119] + checkNoCoalescePartitions(df.sort($"key"), ENSURE_REQUIREMENTS) + } + } + + testGluten("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") { + withTable("t") { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + spark.sql("CREATE TABLE t (c1 int) USING PARQUET") + val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") + assert( + collect(adaptive) { + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + assert(c.hasCoalescedPartition) + c + }.length == 1 + ) + } + } + } + + testGluten("SPARK-35264: Support AQE side broadcastJoin threshold") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldBroadcast: Boolean): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val (origin, adaptive) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin).size == 1) + if (shouldBroadcast) { + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive).size == 1) + } + } + } + + // t1: 1600 bytes + // t2: 160 bytes + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + checkJoinStrategy(false) + } + + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "400") { + checkJoinStrategy(true) + } + } + } + + // table partition size is different with spark + testGluten("SPARK-35264: Support AQE side shuffled hash join formula") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { + size => + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> size) { + val (origin1, adaptive1) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin1).size === 1) + if (shouldShuffleHashJoin && size.toInt < 100000) { + val shj = findTopLevelShuffledHashJoinTransform(adaptive1) + assert(shj.size === 1) + assert(shj.head.joinBuildSide == BuildRight) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive1).size === 1) + } + } + } + // respect user specified join hint + val (origin2, adaptive2) = runAdaptiveAndVerifyResult( + "SELECT /*+ MERGE(t1) */ t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin2).size === 1) + assert(findTopLevelSortMergeJoinTransform(adaptive2).size === 1) + } + + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + // t1 partition size: [395, 316, 313] + // t2 partition size: [140, 50, 0] + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") { + // check default value + checkJoinStrategy(false) + // t1 no hint. + // t2 partition size are all smaller than 200, t2 has SHJ hint. The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "200") { + checkJoinStrategy(true) + } + // t1 no hint. + // Not all partition size of t2 are smaller than 100, t2 no hint. The result is false. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "100") { + checkJoinStrategy(false) + } + // t1, t2 partition size are all smaller than 1000, t1 and t2 can use SHJ. + // The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "1000") { + checkJoinStrategy(true) + } + } + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[ColumnarToCarrierRowExecBase]) + assert( + childPlan + .asInstanceOf[ColumnarToCarrierRowExecBase] + .child + .isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { + Seq("REPARTITION", "REBALANCE(key)") + .foreach { + repartition => + val query = s"SELECT /*+ $repartition */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 1) + assert(!aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + } + + testGluten("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val query = "SELECT /*+ REPARTITION */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 4) + assert(aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + + testGluten("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1" + ) { + + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(if (i > 4) 5 else i, i.toString)), 3) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + def checkPartitionNumber( + query: String, + skewedPartitionNumber: Int, + totalNumber: Int): Unit = { + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } + assert(read.size == 1) + assert( + read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == + skewedPartitionNumber) + assert(read.head.partitionSpecs.size == totalNumber) + } + + // Changed ADVISORY_PARTITION_SIZE_IN_BYTES from 150 to 120 because Gluten has smaller + // partition size. + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "120") { + // partition size [0,208,54,54,54] + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 2, 4) + // partition size [108, 54, 60, 108, 54, 108, 54] + checkPartitionNumber("SELECT /*+ REBALANCE */ * FROM v", 6, 7) + } + + // no skewed partition should be optimized + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10000") { + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 0, 1) + } + } + } + } + + testGluten("SPARK-35888: join with a 0-partition table") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + withTempView("t2") { + // create a temp view with 0 partition + spark + .createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } + assert(aqeReads.length == 2) + aqeReads.foreach { + c => + val stats = c.child.asInstanceOf[QueryStageExec].getRuntimeStatistics + assert(stats.sizeInBytes >= 0) + assert(stats.rowCount.get >= 0) + } + } + } + } + + testGluten("SPARK-35968: AQE coalescing should not produce too small partitions by default") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") + val coalesceRead = collect(adaptive) { + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r + } + assert(coalesceRead.length == 1) + // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. + assert(coalesceRead.head.partitionSpecs.length == 1) + } + } + + testGluten("SPARK-35794: Allow custom plugin for cost evaluator") { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + intercept[IllegalArgumentException] { + CostEvaluator.instantiate( + classOf[InvalidCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.SimpleShuffleSortCostEvaluator") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.InvalidCostEvaluator") { + intercept[IllegalArgumentException] { + runAdaptiveAndVerifyResult(query) + } + } + } + } + + testGluten("SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark + .range(10) + .selectExpr( + "id % 10 as key", + "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", + "array(id, id + 1, id + 3) as c") + .createOrReplaceTempView("t") + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "800") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + val (origin, adaptive) = runAdaptiveAndVerifyResult(query) + assert(findTopLevelSortMergeJoin(origin).size == 1) + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } + } + } + + testGluten( + "SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") { + withTempView("v") { + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + Seq("-1", "10000").foreach { + aqeBhj => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> aqeBhj, + SQLConf.SHUFFLE_PARTITIONS.key -> "1" + ) { + val (origin, adaptive) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM v t1 JOIN ( + | SELECT c1 + 1 as c3 FROM v + |)t2 ON t1.c1 = t2.c3 + |SORT BY c1 + """.stripMargin) + if (aqeBhj.toInt < 0) { + // 1 sort since spark plan has no shuffle for SMJ + assert(findTopLevelSort(origin).size == 1) + // 2 sorts in SMJ + assert(findTopLevelSortTransform(adaptive).size == 2) + } else { + assert(findTopLevelSort(origin).size == 1) + // 1 sort at top node and BHJ has no sort + assert(findTopLevelSortTransform(adaptive).size == 1) + } + } + } + } + } + + testGluten("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1048584", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + // Spark estimates a string column as 20 bytes so with 60k rows, these relations should be + // estimated at ~120m bytes which is greater than the broadcast join threshold. + val joinKeyOne = "00112233445566778899" + val joinKeyTwo = "11223344556677889900" + Seq + .fill(60000)(joinKeyOne) + .toDF("key") + .createOrReplaceTempView("temp") + Seq + .fill(60000)(joinKeyTwo) + .toDF("key") + .createOrReplaceTempView("temp2") + + Seq(joinKeyOne).toDF("key").createOrReplaceTempView("smallTemp") + spark.sql("SELECT key as newKey FROM temp").persist() + + // This query is trying to set up a situation where there are three joins. + // The first join will join the cached relation with a smaller relation. + // The first join is expected to be a broadcast join since the smaller relation will + // fit under the broadcast join threshold. + // The second join will join the first join with another relation and is expected + // to remain as a sort-merge join. + // The third join will join the cached relation with another relation and is expected + // to remain as a sort-merge join. + val query = + s""" + |SELECT t3.newKey + |FROM + | (SELECT t1.newKey + | FROM (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM smallTemp) as t2 + | ON t1.newKey = t2.key + | ) as t3 + | JOIN + | (SELECT key FROM temp2) as t4 + | ON t3.newKey = t4.key + |UNION + |SELECT t1.newKey + |FROM + | (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM temp2) as t2 + | ON t1.newKey = t2.key + |""".stripMargin + val df = spark.sql(query) + df.collect() + val adaptivePlan = df.queryExecution.executedPlan + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.length == 1) + } + } + + testGluten("test log level") { + def verifyLog(expectedLevel: Level): Unit = { + val logAppender = new LogAppender("adaptive execution") + logAppender.setThreshold(expectedLevel) + withLogAppender( + logAppender, + loggerNames = Seq(AdaptiveSparkPlanExec.getClass.getName.dropRight(1)), + level = Some(Level.TRACE)) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + sql("SELECT * FROM testData join testData2 ON key = a where value = '1'").collect() + } + } + Seq("Plan changed", "Final plan").foreach { + msg => + assert(logAppender.loggingEvents.exists { + event => + event.getMessage.getFormattedMessage.contains(msg) && event.getLevel == expectedLevel + }) + } + } + + // Verify default log level + verifyLog(Level.DEBUG) + + // Verify custom log level + val levels = Seq( + "TRACE" -> Level.TRACE, + "trace" -> Level.TRACE, + "DEBUG" -> Level.DEBUG, + "debug" -> Level.DEBUG, + "INFO" -> Level.INFO, + "info" -> Level.INFO, + "WARN" -> Level.WARN, + "warn" -> Level.WARN, + "ERROR" -> Level.ERROR, + "error" -> Level.ERROR, + "deBUG" -> Level.DEBUG + ) + + levels.foreach { + level => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_LOG_LEVEL.key -> level._1) { + verifyLog(level._2) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala new file mode 100644 index 000000000000..4be6d072b4cb --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -0,0 +1,1548 @@ +/* + * 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.spark.sql.execution.adaptive.velox + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ColumnarToCarrierRowExecBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.functions.when +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.logging.log4j.Level + +class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + var finalPlanCnt = 0 + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) { + finalPlanCnt += 1 + } + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + checkAnswer(df, result) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + spark.sparkContext.listenerBus.waitUntilEmpty() + // AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that + // exist out of query stages. + val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1) + assert(finalPlanCnt == expectedFinalPlanCnt) + spark.sparkContext.removeSparkListener(listener) + + val exchanges = adaptivePlan.collect { case e: Exchange => e } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def broadcastHashJoinSize(plan: SparkPlan): Int = { + findTopLevelBroadcastHashJoinTransform(plan).size + findTopLevelBroadcastHashJoin(plan).size + } + + private def findTopLevelBroadcastHashJoinTransform( + plan: SparkPlan): Seq[BroadcastHashJoinExecTransformerBase] = { + collect(plan) { case j: BroadcastHashJoinExecTransformerBase => j } + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { case j: BroadcastHashJoinExec => j } + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { case j: SortMergeJoinExec => j } + } + + private def findTopLevelSortMergeJoinTransform( + plan: SparkPlan): Seq[SortMergeJoinExecTransformer] = { + collect(plan) { case j: SortMergeJoinExecTransformer => j } + } + + private def sortMergeJoinSize(plan: SparkPlan): Int = { + findTopLevelSortMergeJoinTransform(plan).size + findTopLevelSortMergeJoin(plan).size + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { case j: ShuffledHashJoinExec => j } + } + + private def findTopLevelShuffledHashJoinTransform( + plan: SparkPlan): Seq[ShuffledHashJoinExecTransformerBase] = { + collect(plan) { case j: ShuffledHashJoinExecTransformerBase => j } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { case j: BaseJoinExec => j } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { case s: SortExec => s } + } + + private def findTopLevelSortTransform(plan: SparkPlan): Seq[SortExecTransformer] = { + collect(plan) { case s: SortExecTransformer => s } + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { + collectWithSubqueries(plan) { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec, _) => e + } + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } + } + + private def checkNumLocalShuffleReads( + plan: SparkPlan, + numShufflesWithoutLocalRead: Int = 0): Unit = { + val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length + + val numLocalReads = collect(plan) { + case r: AQEShuffleReadExec if r.isLocalRead => r + } + // because columnar local reads cannot execute + numLocalReads.foreach { + r => + val rdd = r.executeColumnar() + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + + testGluten("Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Change broadcast join to merge join") { + withTable("t1", "t2") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + sql("CREATE TABLE t1 USING PARQUET AS SELECT 1 c1") + sql("CREATE TABLE t2 USING PARQUET AS SELECT 1 c1") + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM ( + | SELECT distinct c1 from t1 + | ) tmp1 JOIN ( + | SELECT distinct c1 from t2 + | ) tmp2 ON tmp1.c1 = tmp2.c1 + |""".stripMargin) + assert(broadcastHashJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 0) + assert(findTopLevelSortMergeJoinTransform(adaptivePlan).size == 1) + } + } + } + + testGluten("Reuse the parallelism of coalesced shuffle in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + } + } + + testGluten("Reuse the default parallelism in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + val localShuffleRDD0 = localReads(0) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + val localShuffleRDD1 = localReads(1) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + testGluten("Empty stage coalesced to 1-partition RDD") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + val df1 = spark.range(10).withColumn("a", 'id) + val df2 = spark.range(10).withColumn("b", 'id) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[SortMergeJoinExecTransformer]).isDefined) + } + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformerBase]).isDefined) + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } + assert(coalescedReads.length == 3, s"$plan") + coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) + } + } + } + + testGluten("Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeShuffleWithLocalRead' rule, we can convert all the four + // shuffle read to local shuffle read in the bottom two 'BroadcastHashJoin'. + // For the top level 'BroadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten("multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 2) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + testGluten("Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // // Even with local shuffle read, the query stage reuse can also work. + // gluten change the smj to bhj, stage is changed, so we cannot find the stage with old + // ReuseExchange from stageCache, then the reuse is removed + // https://github.com/apache/spark/pull/24706/ + // files#diff-ec42cd27662f3f528832c298a60fffa1d341feb04aa1d8c80044b70cbe0ebbfcR224 + // maybe vanilla spark should checkReuse rile again + // val ex = findReusedExchange(adaptivePlan) + // assert(ex.size == 1) + } + } + + testGluten("Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + testGluten("Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + testGluten("Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.child.isInstanceOf[ColumnarBroadcastExchangeExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + // Cost is equal, not test cost is greater, need new test, but other test may contain cost change, + // so it maybe not essential + testGluten("Avoid plan change if cost is greater") {} + + testGluten("Change merge join to broadcast join without local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + assert(sortMergeJoinSize(plan) == 2) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten( + "Avoid changing merge join to broadcast join if too many empty partitions " + + "on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5", + // this config will make some empty partitions + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + // because testData2 in gluten sizeInBytes(from ColumnarShuffleExchangeExec plan stats) + // is 24B sometimes, so change the threshold from 80 to 20 + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.joinBuildSide == BuildRight) + } + } + } + + testGluten("SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "100", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 3 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(query: String, optimizeSkewJoin: Boolean): Unit = { + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult(query) + val innerSmj = findTopLevelSortMergeJoinTransform(innerAdaptivePlan) + assert(innerSmj.size == 1 && innerSmj.head.isSkewJoin == optimizeSkewJoin) + } + + // OptimizeSkewedJoin check the map status, because the + checkSkewJoin("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2", true) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + checkSkewJoin( + "SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 GROUP BY key1", + false) + } + } + } + + testGluten("SPARK-29544: adaptive skew join with different join types") { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { + joinHint => + def getJoinNode(plan: SparkPlan): Seq[BinaryExecNode] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoinTransform(plan) + } else { + findTopLevelShuffledHashJoinTransform(plan) + } + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "800" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + + def checkSkewJoin( + joins: Seq[BinaryExecNode], + leftSkewNum: Int, + rightSkewNum: Int): Unit = { + assert(joins.size == 1) + joins.head match { + case s: SortMergeJoinExecTransformer => assert(s.isSkewJoin) + case g: ShuffledHashJoinExecTransformerBase => assert(g.isSkewJoin) + case _ => assert(false) + } + assert( + joins.head.left + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == leftSkewNum) + assert( + joins.head.right + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == rightSkewNum) + } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "JOIN skewData2 ON key1 = key2") + val inner = getJoinNode(innerAdaptivePlan) + // checkSkewJoin(inner, 2, 1) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData2) */ * FROM skewData1 " + + "LEFT OUTER JOIN skewData2 ON key1 = key2") + val leftJoin = getJoinNode(leftAdaptivePlan) + // checkSkewJoin(leftJoin, 2, 0) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "RIGHT OUTER JOIN skewData2 ON key1 = key2") + val rightJoin = getJoinNode(rightAdaptivePlan) + // checkSkewJoin(rightJoin, 0, 1) + } + } + } + } + + testGluten("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + + testGluten("metrics of the shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(!read.isLocalRead) + assert(!read.hasSkewedPartition) + assert(read.hasCoalescedPartition) + assert( + read.metrics.keys.toSeq.sorted == Seq( + "numCoalescedPartitions", + "numPartitions", + "partitionDataSize")) + assert(read.metrics("numCoalescedPartitions").value == 1) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + assert(read.metrics("partitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformerBase => j }.head + assert(join.joinBuildSide == BuildLeft) + + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(read.isLocalRead) + assert(read.metrics.keys.toSeq == Seq("numPartitions")) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1000" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + } + } + } + } + + // because gluten use columnar format, which cannot execute to get rowIterator, then get the key + // null status + ignore( + GLUTEN_TEST + "SPARK-32573: Eliminate NAAJ when BuildSide is HashedRelationWithAllNullKeys") {} + + // EmptyRelation case + ignore( + GLUTEN_TEST + "SPARK-35455: Unify empty relation optimization " + + "between normal and AQE optimizer - single join") {} + + testGluten("SPARK-32753: Only copy tags to node with no tags") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("v1") { + spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1") + + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { case s: ColumnarShuffleExchangeExec => s }.length == 1) + } + } + } + + testGluten("Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + // this test default level is WARN, so we should check warn level + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "WARN", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80" + ) { + sql( + "SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq( + "=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===" + ).foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg))) + } + } + } + + testGluten("SPARK-33551: Do not use AQE shuffle read for repartition") { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => + s.shuffleOrigin == REPARTITION_BY_COL || s.shuffleOrigin == REPARTITION_BY_NUM + case _ => false + }.isDefined + } + + def checkBHJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + probeSideLocalRead: Boolean, + probeSideCoalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + // There should be only one shuffle that can't do local read, which is either the top shuffle + // from repartition, or BHJ probe side shuffle. + checkNumLocalShuffleReads(plan, 1) + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val bhj = findTopLevelBroadcastHashJoinTransform(plan) + assert(bhj.length == 1) + + // Build side should do local read. + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) + assert(buildSide.isDefined) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) + if (probeSideLocalRead || probeSideCoalescedRead) { + assert(probeSide.isDefined) + if (probeSideLocalRead) { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + } else { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) + } + } else { + assert(probeSide.isEmpty) + } + } + + def checkSMJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + optimizeSkewJoin: Boolean, + coalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.length == 1) + assert(smj.head.isSkewJoin == optimizeSkewJoin) + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } + if (coalescedRead || optimizeSkewJoin) { + assert(aqeReads.length == 2) + if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) + } else { + assert(aqeReads.isEmpty) + } + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val df = sql(""" + |SELECT * FROM ( + | SELECT * FROM testData WHERE key = 1 + |) + |RIGHT OUTER JOIN testData2 + |ON value = b + """.stripMargin) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + // Repartition with no partition num specified. + checkBHJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkBHJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out + // The final plan must have 5 partitions, no optimization can be made to the probe side. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkBHJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + + // Repartition by col and project away the partition cols + checkBHJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + } + + // Force skew join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_ENABLED.key -> "true", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR.key -> "0", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10" + ) { + // Repartition with no partition num specified. + checkSMJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = true) + + // Repartition with default partition num (5 in test env) specified. + checkSMJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out. + // The final plan must have 5 partitions, can't do coalesced read. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkSMJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false) + + // Repartition by col and project away the partition cols + checkSMJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + } + } + } + + testGluten("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "true") { + withTable("t1") { + spark.range(100).selectExpr("id + 1 as a").write.format("parquet").saveAsTable("t1") + val query = "SELECT SUM(a) FROM t1 GROUP BY a" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val metricName = SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED + val blocksFetchedMetric = collectFirst(adaptivePlan) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric.isDefined) + val blocksFetched = blocksFetchedMetric.get.value + withSQLConf(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "false") { + val (_, adaptivePlan2) = runAdaptiveAndVerifyResult(query) + val blocksFetchedMetric2 = collectFirst(adaptivePlan2) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric2.isDefined) + val blocksFetched2 = blocksFetchedMetric2.get.value + assert(blocksFetched == blocksFetched2) + } + } + } + } + + testGluten("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") { + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan + assert(collect(plan) { + case s: ColumnarShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => + s + }.size == 1) + checkAnswer(ds, testData) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + // Pick a small value so that no coalesce can happen. + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2" + ) { + val df = + spark.sparkContext.parallelize((1 to 100).map(i => TestData(i, i.toString)), 10).toDF() + + // partition size [1420, 1420] + checkNoCoalescePartitions(df.repartition($"key"), REPARTITION_BY_COL) + // partition size [1140, 1119] + checkNoCoalescePartitions(df.sort($"key"), ENSURE_REQUIREMENTS) + } + } + + testGluten("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") { + withTable("t") { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + spark.sql("CREATE TABLE t (c1 int) USING PARQUET") + val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") + assert( + collect(adaptive) { + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + assert(c.hasCoalescedPartition) + c + }.length == 1 + ) + } + } + } + + testGluten("SPARK-35264: Support AQE side broadcastJoin threshold") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldBroadcast: Boolean): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val (origin, adaptive) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin).size == 1) + if (shouldBroadcast) { + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive).size == 1) + } + } + } + + // t1: 1600 bytes + // t2: 160 bytes + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + checkJoinStrategy(false) + } + + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "400") { + checkJoinStrategy(true) + } + } + } + + // table partition size is different with spark + testGluten("SPARK-35264: Support AQE side shuffled hash join formula") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { + size => + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> size) { + val (origin1, adaptive1) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin1).size === 1) + if (shouldShuffleHashJoin && size.toInt < 100000) { + val shj = findTopLevelShuffledHashJoinTransform(adaptive1) + assert(shj.size === 1) + assert(shj.head.joinBuildSide == BuildRight) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive1).size === 1) + } + } + } + // respect user specified join hint + val (origin2, adaptive2) = runAdaptiveAndVerifyResult( + "SELECT /*+ MERGE(t1) */ t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin2).size === 1) + assert(findTopLevelSortMergeJoinTransform(adaptive2).size === 1) + } + + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + // t1 partition size: [395, 316, 313] + // t2 partition size: [140, 50, 0] + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") { + // check default value + checkJoinStrategy(false) + // t1 no hint. + // t2 partition size are all smaller than 200, t2 has SHJ hint. The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "200") { + checkJoinStrategy(true) + } + // t1 no hint. + // Not all partition size of t2 are smaller than 100, t2 no hint. The result is false. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "100") { + checkJoinStrategy(false) + } + // t1, t2 partition size are all smaller than 1000, t1 and t2 can use SHJ. + // The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "1000") { + checkJoinStrategy(true) + } + } + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[ColumnarToCarrierRowExecBase]) + assert( + childPlan + .asInstanceOf[ColumnarToCarrierRowExecBase] + .child + .isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { + Seq("REPARTITION", "REBALANCE(key)") + .foreach { + repartition => + val query = s"SELECT /*+ $repartition */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 1) + assert(!aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + } + + testGluten("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val query = "SELECT /*+ REPARTITION */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 4) + assert(aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + + testGluten("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1" + ) { + + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(if (i > 4) 5 else i, i.toString)), 3) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + def checkPartitionNumber( + query: String, + skewedPartitionNumber: Int, + totalNumber: Int): Unit = { + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } + assert(read.size == 1) + assert( + read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == + skewedPartitionNumber) + assert(read.head.partitionSpecs.size == totalNumber) + } + + // Changed ADVISORY_PARTITION_SIZE_IN_BYTES from 150 to 120 because Gluten has smaller + // partition size. + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "120") { + // partition size [0,208,54,54,54] + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 2, 4) + // partition size [108, 54, 60, 108, 54, 108, 54] + checkPartitionNumber("SELECT /*+ REBALANCE */ * FROM v", 6, 7) + } + + // no skewed partition should be optimized + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10000") { + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 0, 1) + } + } + } + } + + testGluten("SPARK-35888: join with a 0-partition table") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + withTempView("t2") { + // create a temp view with 0 partition + spark + .createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } + assert(aqeReads.length == 2) + aqeReads.foreach { + c => + val stats = c.child.asInstanceOf[QueryStageExec].getRuntimeStatistics + assert(stats.sizeInBytes >= 0) + assert(stats.rowCount.get >= 0) + } + } + } + } + + testGluten("SPARK-35968: AQE coalescing should not produce too small partitions by default") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") + val coalesceRead = collect(adaptive) { + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r + } + assert(coalesceRead.length == 1) + // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. + assert(coalesceRead.head.partitionSpecs.length == 1) + } + } + + testGluten("SPARK-35794: Allow custom plugin for cost evaluator") { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + intercept[IllegalArgumentException] { + CostEvaluator.instantiate( + classOf[InvalidCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.SimpleShuffleSortCostEvaluator") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.InvalidCostEvaluator") { + intercept[IllegalArgumentException] { + runAdaptiveAndVerifyResult(query) + } + } + } + } + + testGluten("SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark + .range(10) + .selectExpr( + "id % 10 as key", + "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", + "array(id, id + 1, id + 3) as c") + .createOrReplaceTempView("t") + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "800") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + val (origin, adaptive) = runAdaptiveAndVerifyResult(query) + assert(findTopLevelSortMergeJoin(origin).size == 1) + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } + } + } + + testGluten( + "SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") { + withTempView("v") { + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + Seq("-1", "10000").foreach { + aqeBhj => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> aqeBhj, + SQLConf.SHUFFLE_PARTITIONS.key -> "1" + ) { + val (origin, adaptive) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM v t1 JOIN ( + | SELECT c1 + 1 as c3 FROM v + |)t2 ON t1.c1 = t2.c3 + |SORT BY c1 + """.stripMargin) + if (aqeBhj.toInt < 0) { + // 1 sort since spark plan has no shuffle for SMJ + assert(findTopLevelSort(origin).size == 1) + // 2 sorts in SMJ + assert(findTopLevelSortTransform(adaptive).size == 2) + } else { + assert(findTopLevelSort(origin).size == 1) + // 1 sort at top node and BHJ has no sort + assert(findTopLevelSortTransform(adaptive).size == 1) + } + } + } + } + } + + testGluten("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1048584", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + // Spark estimates a string column as 20 bytes so with 60k rows, these relations should be + // estimated at ~120m bytes which is greater than the broadcast join threshold. + val joinKeyOne = "00112233445566778899" + val joinKeyTwo = "11223344556677889900" + Seq + .fill(60000)(joinKeyOne) + .toDF("key") + .createOrReplaceTempView("temp") + Seq + .fill(60000)(joinKeyTwo) + .toDF("key") + .createOrReplaceTempView("temp2") + + Seq(joinKeyOne).toDF("key").createOrReplaceTempView("smallTemp") + spark.sql("SELECT key as newKey FROM temp").persist() + + // This query is trying to set up a situation where there are three joins. + // The first join will join the cached relation with a smaller relation. + // The first join is expected to be a broadcast join since the smaller relation will + // fit under the broadcast join threshold. + // The second join will join the first join with another relation and is expected + // to remain as a sort-merge join. + // The third join will join the cached relation with another relation and is expected + // to remain as a sort-merge join. + val query = + s""" + |SELECT t3.newKey + |FROM + | (SELECT t1.newKey + | FROM (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM smallTemp) as t2 + | ON t1.newKey = t2.key + | ) as t3 + | JOIN + | (SELECT key FROM temp2) as t4 + | ON t3.newKey = t4.key + |UNION + |SELECT t1.newKey + |FROM + | (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM temp2) as t2 + | ON t1.newKey = t2.key + |""".stripMargin + val df = spark.sql(query) + df.collect() + val adaptivePlan = df.queryExecution.executedPlan + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.length == 1) + } + } + + testGluten("test log level") { + def verifyLog(expectedLevel: Level): Unit = { + val logAppender = new LogAppender("adaptive execution") + logAppender.setThreshold(expectedLevel) + withLogAppender( + logAppender, + loggerNames = Seq(AdaptiveSparkPlanExec.getClass.getName.dropRight(1)), + level = Some(Level.TRACE)) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + sql("SELECT * FROM testData join testData2 ON key = a where value = '1'").collect() + } + } + Seq("Plan changed", "Final plan").foreach { + msg => + assert(logAppender.loggingEvents.exists { + event => + event.getMessage.getFormattedMessage.contains(msg) && event.getLevel == expectedLevel + }) + } + } + + // Verify default log level + verifyLog(Level.DEBUG) + + // Verify custom log level + val levels = Seq( + "TRACE" -> Level.TRACE, + "trace" -> Level.TRACE, + "DEBUG" -> Level.DEBUG, + "debug" -> Level.DEBUG, + "INFO" -> Level.INFO, + "info" -> Level.INFO, + "WARN" -> Level.WARN, + "warn" -> Level.WARN, + "ERROR" -> Level.ERROR, + "error" -> Level.ERROR, + "deBUG" -> Level.DEBUG + ) + + levels.foreach { + level => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_LOG_LEVEL.key -> level._1) { + verifyLog(level._2) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala new file mode 100644 index 000000000000..12cc721154a0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -0,0 +1,227 @@ +/* + * 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.spark.sql.execution.benchmarks + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} +import org.apache.gluten.extension.columnar.transition.Transitions +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import scala.collection.JavaConverters._ + +/** + * Benchmark to measure native parquet read performance. To run this benchmark: + * {{{ + * 1. Run in IDEA: run this class directly; + * 2. Run without IDEA: bin/spark-submit --class + * --jars ,, + * --conf xxxx=xxx + * gluten-ut-XXX-tests.jar + * parameters + * + * Parameters: + * 1. parquet files dir; + * 2. the fields to read; + * 3. the execution count; + * 4. whether to run vanilla spark benchmarks; + * }}} + */ +object ParquetReadBenchmark extends SqlBasedBenchmark { + + protected lazy val thrdNum = "1" + protected lazy val memorySize = "4G" + protected lazy val offheapSize = "4G" + + def beforeAll(): Unit = {} + + override def getSparkSession: SparkSession = { + beforeAll(); + val conf = new SparkConf() + .setAppName("ParquetReadBenchmark") + .setIfMissing("spark.master", s"local[$thrdNum]") + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .setIfMissing("spark.memory.offHeap.size", offheapSize) + .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") + .set("spark.sql.adaptive.enabled", "false") + .setIfMissing("spark.driver.memory", memorySize) + .setIfMissing("spark.executor.memory", memorySize) + .setIfMissing("spark.sql.files.maxPartitionBytes", "1G") + .setIfMissing("spark.sql.files.openCostInBytes", "1073741824") + + if (BackendTestUtils.isCHBackendLoaded()) { + conf + .set("spark.io.compression.codec", "LZ4") + .set(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") + .set( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") + .set("spark.databricks.delta.maxSnapshotLineageLength", "20") + .set("spark.databricks.delta.snapshotPartitions", "1") + .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") + .set("spark.databricks.delta.stalenessLimit", "3600000") + } + + SparkSession.builder.config(conf).getOrCreate() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val (parquetDir, scanSchema, executedCnt, executedVanilla) = + if (mainArgs.isEmpty) { + ("/data/tpch-data-sf10/lineitem", "l_orderkey,l_receiptdate", 5, true) + } else { + (mainArgs(0), mainArgs(1), mainArgs(2).toInt, mainArgs(3).toBoolean) + } + + val parquetReadDf = spark.sql(s""" + |select $scanSchema from parquet.`$parquetDir` + | + |""".stripMargin) + // Get the `FileSourceScanExecTransformer` + val fileScan = parquetReadDf.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val filePartitions = fileScan.getPartitions + .map(_.asInstanceOf[FilePartition]) + + val wholeStageTransform = parquetReadDf.queryExecution.executedPlan.collect { + case wholeStage: WholeStageTransformer => wholeStage + }.head + + // remove ProjectExecTransformer + val newWholeStage = wholeStageTransform.withNewChildren(Seq(fileScan)) + + // generate ColumnarToRow + val columnarToRowPlan = Transitions.toRowPlan(newWholeStage) + + val newWholeStageRDD = newWholeStage.executeColumnar() + val newColumnarToRowRDD = columnarToRowPlan.execute() + + // Get the total row count + val totalRowCnt = newWholeStageRDD + .mapPartitionsInternal( + batches => { + batches.map(batch => batch.numRows().toLong) + }) + .collect() + .sum + + val parquetReadBenchmark = + new Benchmark( + s"Parquet Read files, fields: $scanSchema, total $totalRowCnt records", + totalRowCnt, + output = output) + + parquetReadBenchmark.addCase(s"Native Parquet Read", executedCnt) { + _ => + val resultRDD: RDD[Long] = newWholeStageRDD.mapPartitionsInternal { + batches => + batches.foreach(batch => batch.numRows().toLong) + Iterator.empty + } + resultRDD.collect() + } + + parquetReadBenchmark.addCase(s"Native Parquet Read to Rows", executedCnt) { + _ => + val resultRDD: RDD[Int] = newColumnarToRowRDD.mapPartitionsInternal { + rows => + rows.foreach(_.numFields) + Iterator.empty + } + resultRDD.collect() + } + + if (executedVanilla) { + spark.conf.set(GlutenConfig.GLUTEN_ENABLED.key, "false") + + val vanillaParquet = spark.sql(s""" + |select $scanSchema from parquet.`$parquetDir` + | + |""".stripMargin) + + val vanillaScanPlan = vanillaParquet.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan + } + + val fileScan = vanillaScanPlan.head + val fileScanOutput = fileScan.output + val relation = fileScan.relation + val readFile: (PartitionedFile) => Iterator[InternalRow] = + relation.fileFormat.buildReaderWithPartitionValues( + sparkSession = relation.sparkSession, + dataSchema = relation.dataSchema, + partitionSchema = relation.partitionSchema, + requiredSchema = fileScan.requiredSchema, + filters = Seq.empty, + options = relation.options, + hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) + ) + + val newFileScanRDD = new FileScanRDD(spark, readFile, filePartitions, fileScan.requiredSchema) + .asInstanceOf[RDD[ColumnarBatch]] + + val rowCnt = newFileScanRDD + .mapPartitionsInternal(batches => batches.map(batch => batch.numRows().toLong)) + .collect() + .sum + assert(totalRowCnt == rowCnt, "The row count of the benchmark is not equal.") + + parquetReadBenchmark.addCase(s"Vanilla Spark Parquet Read", executedCnt) { + _ => + val resultRDD: RDD[Long] = newFileScanRDD.mapPartitionsInternal { + batches => + batches.foreach(_.numRows().toLong) + Iterator.empty + } + resultRDD.collect() + } + + parquetReadBenchmark.addCase(s"Vanilla Spark Parquet Read to Rows", executedCnt) { + _ => + val resultRDD: RDD[Long] = newFileScanRDD.mapPartitionsInternal { + batches => + val toUnsafe = UnsafeProjection.create(fileScanOutput, fileScanOutput) + batches.foreach(_.rowIterator().asScala.map(toUnsafe).foreach(_.numFields)) + Iterator.empty + } + resultRDD.collect() + } + } + + parquetReadBenchmark.run() + } + + override def afterAll(): Unit = { + super.afterAll() + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala new file mode 100644 index 000000000000..37a786e34c53 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketingUtilsSuite extends BucketingUtilsSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala new file mode 100644 index 000000000000..eeb63436c1e1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceStrategySuite extends DataSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala new file mode 100644 index 000000000000..6435d17de2ab --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceSuite extends DataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala new file mode 100644 index 000000000000..c0ba24f2be1f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala @@ -0,0 +1,25 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + +class GlutenFileFormatWriterSuite + extends FileFormatWriterSuite + with GlutenSQLTestsBaseTrait + with CodegenInterpretedPlanTest {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala new file mode 100644 index 000000000000..c1c57eaa9145 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileIndexSuite extends FileIndexSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala new file mode 100644 index 000000000000..ed347d024c1c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala @@ -0,0 +1,159 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{FileSourceScanExecTransformer, FilterExecTransformer} + +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} + +import java.io.File +import java.sql.Timestamp + +import scala.reflect.ClassTag + +class GlutenFileMetadataStructSuite extends FileMetadataStructSuite with GlutenSQLTestsBaseTrait { + + val schemaWithFilePathField: StructType = new StructType() + .add(StructField("file_path", StringType)) + .add(StructField("age", IntegerType)) + .add( + StructField( + "info", + new StructType() + .add(StructField("id", LongType)) + .add(StructField("university", StringType)))) + + private val METADATA_FILE_PATH = "_metadata.file_path" + private val METADATA_FILE_NAME = "_metadata.file_name" + private val METADATA_FILE_SIZE = "_metadata.file_size" + private val METADATA_FILE_MODIFICATION_TIME = "_metadata.file_modification_time" + + private def getMetadataForFile(f: File): Map[String, Any] = { + Map( + METADATA_FILE_PATH -> f.toURI.toString, + METADATA_FILE_NAME -> f.getName, + METADATA_FILE_SIZE -> f.length(), + METADATA_FILE_MODIFICATION_TIME -> new Timestamp(f.lastModified()) + ) + } + + private def metadataColumnsNativeTest(testName: String, fileSchema: StructType)( + f: (DataFrame, Map[String, Any], Map[String, Any]) => Unit): Unit = { + Seq("parquet").foreach { + testFileFormat => + testGluten(s"metadata struct ($testFileFormat): " + testName) { + withTempDir { + dir => + import scala.collection.JavaConverters._ + + // 1. create df0 and df1 and save under /data/f0 and /data/f1 + val df0 = spark.createDataFrame(data0.asJava, fileSchema) + val f0 = new File(dir, "data/f0").getCanonicalPath + df0.coalesce(1).write.format(testFileFormat).save(f0) + + val df1 = spark.createDataFrame(data1.asJava, fileSchema) + val f1 = new File(dir, "data/f1 gluten").getCanonicalPath + df1.coalesce(1).write.format(testFileFormat).save(f1) + + // 2. read both f0 and f1 + val df = spark.read + .format(testFileFormat) + .schema(fileSchema) + .load(new File(dir, "data").getCanonicalPath + "/*") + val realF0 = new File(dir, "data/f0") + .listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")) + .head + val realF1 = new File(dir, "data/f1 gluten") + .listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")) + .head + f(df, getMetadataForFile(realF0), getMetadataForFile(realF1)) + } + } + } + } + + def checkOperatorMatch[T](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass)) + } + + metadataColumnsNativeTest( + "plan check with metadata and user data select", + schemaWithFilePathField) { + (df, f0, f1) => + var dfWithMetadata = df.select( + METADATA_FILE_NAME, + METADATA_FILE_PATH, + METADATA_FILE_SIZE, + METADATA_FILE_MODIFICATION_TIME, + "age") + dfWithMetadata.collect + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](dfWithMetadata) + } else { + checkOperatorMatch[FileSourceScanExec](dfWithMetadata) + } + + // would fallback + dfWithMetadata = df.select(METADATA_FILE_PATH, "file_path") + checkAnswer( + dfWithMetadata, + Seq( + Row(f0(METADATA_FILE_PATH), "jack"), + Row(f1(METADATA_FILE_PATH), "lily") + ) + ) + checkOperatorMatch[FileSourceScanExec](dfWithMetadata) + } + + metadataColumnsNativeTest("plan check with metadata filter", schemaWithFilePathField) { + (df, f0, f1) => + var filterDF = df + .select("file_path", "age", METADATA_FILE_NAME) + .where(Column(METADATA_FILE_NAME) === f0((METADATA_FILE_NAME))) + val ret = filterDF.collect + assert(ret.size == 1) + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](filterDF) + } else { + checkOperatorMatch[FileSourceScanExec](filterDF) + } + checkOperatorMatch[FilterExecTransformer](filterDF) + + // case to check if file_path is URI string + filterDF = + df.select(METADATA_FILE_PATH).where(Column(METADATA_FILE_NAME) === f1((METADATA_FILE_NAME))) + checkAnswer( + filterDF, + Seq( + Row(f1(METADATA_FILE_PATH)) + ) + ) + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](filterDF) + } else { + checkOperatorMatch[FileSourceScanExec](filterDF) + } + checkOperatorMatch[FilterExecTransformer](filterDF) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala new file mode 100644 index 000000000000..54138564f95f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.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 org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetV1AggregatePushDownSuite + extends ParquetV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetV2AggregatePushDownSuite + extends ParquetV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1AggregatePushDownSuite + extends OrcV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV2AggregatePushDownSuite + extends OrcV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala new file mode 100644 index 000000000000..631be9c96fa9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetCodecSuite extends ParquetCodecSuite with GlutenSQLTestsBaseTrait {} + +class GlutenOrcCodecSuite extends OrcCodecSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala new file mode 100644 index 000000000000..9aed8f6d6541 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileSourceCustomMetadataStructSuite + extends FileSourceCustomMetadataStructSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala new file mode 100644 index 000000000000..171a27e31c47 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql._ + +class GlutenFileSourceStrategySuite extends FileSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala new file mode 100644 index 000000000000..b283d44b03a4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenHadoopFileLinesReaderSuite + extends HadoopFileLinesReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala new file mode 100644 index 000000000000..f3554eb1cb09 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterStrategySuite extends PathFilterStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala new file mode 100644 index 000000000000..4f4f9c76ee4a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterSuite extends PathFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala new file mode 100644 index 000000000000..a108c4fe1ecf --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPruneFileSourcePartitionsSuite + extends PruneFileSourcePartitionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala new file mode 100644 index 000000000000..6ea7dd910194 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala @@ -0,0 +1,155 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf + +import java.io.File + +class GlutenCSVReadSchemaSuite extends CSVReadSchemaSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") +} + +class GlutenHeaderCSVReadSchemaSuite extends HeaderCSVReadSchemaSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") +} + +class GlutenJsonReadSchemaSuite extends JsonReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenOrcReadSchemaSuite extends OrcReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedOrcReadSchemaSuite + extends VectorizedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + private lazy val values = 1 to 10 + private lazy val floatDF = values.map(_.toFloat).toDF("col1") + private lazy val doubleDF = values.map(_.toDouble).toDF("col1") + private lazy val unionDF = floatDF.union(doubleDF) + + testGluten("change column position") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val df1 = Seq(("1", "a"), ("2", "b"), ("3", "c")).toDF("col1", "col2") + val df2 = Seq(("d", "4"), ("e", "5"), ("f", "6")).toDF("col2", "col1") + val unionDF = df1.unionByName(df2) + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1", "col2") + + checkAnswer(df, unionDF) + } + } + } + + testGluten("read byte, int, short, long together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val byteDF = (Byte.MaxValue - 2 to Byte.MaxValue).map(_.toByte).toDF("col1") + val shortDF = (Short.MaxValue - 2 to Short.MaxValue).map(_.toShort).toDF("col1") + val intDF = (Int.MaxValue - 2 to Int.MaxValue).toDF("col1") + val longDF = (Long.MaxValue - 2 to Long.MaxValue).toDF("col1") + val unionDF = byteDF.union(shortDF).union(intDF).union(longDF) + + val byteDir = s"$path${File.separator}part=byte" + val shortDir = s"$path${File.separator}part=short" + val intDir = s"$path${File.separator}part=int" + val longDir = s"$path${File.separator}part=long" + + byteDF.write.format(format).options(options).save(byteDir) + shortDF.write.format(format).options(options).save(shortDir) + intDF.write.format(format).options(options).save(intDir) + longDF.write.format(format).options(options).save(longDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } + + testGluten("read float and double together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val floatDir = s"$path${File.separator}part=float" + val doubleDir = s"$path${File.separator}part=double" + + floatDF.write.format(format).options(options).save(floatDir) + doubleDF.write.format(format).options(options).save(doubleDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } +} + +class GlutenMergedOrcReadSchemaSuite + extends MergedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetReadSchemaSuite extends ParquetReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedParquetReadSchemaSuite + extends VectorizedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenMergedParquetReadSchemaSuite + extends MergedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala new file mode 100644 index 000000000000..879b0badf1dd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTableLocationSuite extends TableLocationSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala new file mode 100644 index 000000000000..a287f5fffb62 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala @@ -0,0 +1,268 @@ +/* + * 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.spark.sql.execution.datasources + +import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} +import org.apache.spark.sql.execution.{QueryExecution, SortExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.util.QueryExecutionListener + +trait GlutenV1WriteCommandSuiteBase extends V1WriteCommandSuiteBase { + + override def beforeAll(): Unit = { + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + } + + override def executeAndCheckOrdering( + hasLogicalSort: Boolean, + orderingMatched: Boolean, + hasEmpty2Null: Boolean = false)(query: => Unit): Unit = { + var optimizedPlan: LogicalPlan = null + + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.optimizedPlan match { + case w: V1WriteCommand => + if (hasLogicalSort && conf.getConf(SQLConf.PLANNED_WRITE_ENABLED)) { + assert(w.query.isInstanceOf[WriteFiles]) + assert(w.partitionColumns == w.query.asInstanceOf[WriteFiles].partitionColumns) + optimizedPlan = w.query.asInstanceOf[WriteFiles].child + } else { + optimizedPlan = w.query + } + case _ => + } + } + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + query + + // RemoveNativeWriteFilesSortAndProject remove SortExec or SortExecTransformer, + // thus, FileFormatWriter.outputOrderingMatched is false. + if (!conf.getConf(SQLConf.PLANNED_WRITE_ENABLED)) { + // Check whether the output ordering is matched before FileFormatWriter executes rdd. + assert( + FileFormatWriter.outputOrderingMatched == orderingMatched, + s"Expect: $orderingMatched, Actual: ${FileFormatWriter.outputOrderingMatched}") + } + + sparkContext.listenerBus.waitUntilEmpty() + + assert(optimizedPlan != null) + // Check whether exists a logical sort node of the write query. + // If user specified sort matches required ordering, the sort node may not at the top of query. + assert( + optimizedPlan.exists(_.isInstanceOf[Sort]) == hasLogicalSort, + s"Expect hasLogicalSort: $hasLogicalSort," + + s"Actual: ${optimizedPlan.exists(_.isInstanceOf[Sort])}" + ) + + // Check empty2null conversion. + val empty2nullExpr = optimizedPlan.exists(p => V1WritesUtils.hasEmptyToNull(p.expressions)) + assert( + empty2nullExpr == hasEmpty2Null, + s"Expect hasEmpty2Null: $hasEmpty2Null, Actual: $empty2nullExpr. Plan:\n$optimizedPlan") + + spark.listenerManager.unregister(listener) + } +} + +class GlutenV1WriteCommandSuite + extends V1WriteCommandSuite + with GlutenV1WriteCommandSuiteBase + with GlutenSQLTestsBaseTrait + with GlutenColumnarWriteTestSupport { + + // TODO: fix in Spark-4.0 + ignoreGluten( + "SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition column") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withPlannedWrite { + enabled => + withTable("t") { + sql(""" + |CREATE TABLE t(b INT, value STRING) USING PARQUET + |PARTITIONED BY (key INT) + |""".stripMargin) + executeAndCheckOrdering(hasLogicalSort = true, orderingMatched = true) { + sql(""" + |INSERT INTO t + |SELECT b, value, key + |FROM testData JOIN testData2 ON key = a + |SORT BY key, value + |""".stripMargin) + } + + // inspect the actually executed plan (that is different to executeAndCheckOrdering) + assert(FileFormatWriter.executedPlan.isDefined) + val executedPlan = FileFormatWriter.executedPlan.get + + val plan = if (enabled) { + checkWriteFilesAndGetChild(executedPlan) + } else { + executedPlan.transformDown { case a: AdaptiveSparkPlanExec => a.executedPlan } + } + + // assert the outer most sort in the executed plan + assert( + plan + .collectFirst { + case s: SortExec => s + case ns: SortExecTransformer => ns + } + .exists { + case SortExec( + Seq( + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case SortExecTransformer( + Seq( + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case _ => false + }, + plan + ) + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-41914: v1 write with AQE and in-partition sorted - string partition column") { + withPlannedWrite { + enabled => + withTable("t") { + sql(""" + |CREATE TABLE t(key INT, b INT) USING PARQUET + |PARTITIONED BY (value STRING) + |""".stripMargin) + executeAndCheckOrdering( + hasLogicalSort = true, + orderingMatched = true, + hasEmpty2Null = enabled) { + sql(""" + |INSERT INTO t + |SELECT key, b, value + |FROM testData JOIN testData2 ON key = a + |SORT BY value, key + |""".stripMargin) + } + + // inspect the actually executed plan (that is different to executeAndCheckOrdering) + assert(FileFormatWriter.executedPlan.isDefined) + val executedPlan = FileFormatWriter.executedPlan.get + + val plan = if (enabled) { + checkWriteFilesAndGetChild(executedPlan) + } else { + executedPlan.transformDown { case a: AdaptiveSparkPlanExec => a.executedPlan } + } + + // assert the outer most sort in the executed plan + assert( + plan + .collectFirst { + case s: SortExec => s + case ns: SortExecTransformer => ns + } + .exists { + case SortExec( + Seq( + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case SortExecTransformer( + Seq( + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case _ => false + }, + plan + ) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala new file mode 100644 index 000000000000..ee6ec1bea1af --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources.binaryfile + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBinaryFileFormatSuite extends BinaryFileFormatSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala new file mode 100644 index 000000000000..6cfa9f2028e8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala @@ -0,0 +1,135 @@ +/* + * 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.spark.sql.execution.datasources.csv + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.exception.GlutenException + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DateType, IntegerType, StructType, TimestampType} + +import org.scalatest.exceptions.TestFailedException + +import java.sql.{Date, Timestamp} + +class GlutenCSVSuite extends CSVSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenCSVv1Suite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "csv") +} + +class GlutenCSVv2Suite extends GlutenCSVSuite { + + import testImplicits._ + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + + override def testNameBlackList: Seq[String] = Seq( + // overwritten with different test + "test for FAILFAST parsing mode", + "SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern" + ) + + testGluten("test for FAILFAST parsing mode") { + Seq(false, true).foreach { + multiLine => + val exception = intercept[SparkException] { + spark.read + .format("csv") + .option("multiLine", multiLine) + .options(Map("header" -> "true", "mode" -> "failfast")) + .load(testFile(carsFile)) + .collect() + } + + assert(exception.getCause.isInstanceOf[GlutenException]) + assert( + exception.getMessage.contains( + "[MALFORMED_RECORD_IN_PARSING] Malformed records are detected in record parsing: " + + "[2015,Chevy,Volt,null,null]")) + } + } + + testGluten("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") { + withTempPath { + path => + Seq("1,2020011,2020011", "2,20201203,20201203") + .toDF() + .repartition(1) + .write + .text(path.getAbsolutePath) + val schema = new StructType() + .add("id", IntegerType) + .add("date", DateType) + .add("ts", TimestampType) + val output = spark.read + .schema(schema) + .option("dateFormat", "yyyyMMdd") + .option("timestampFormat", "yyyyMMdd") + .csv(path.getAbsolutePath) + + def check(mode: String, res: Seq[Row]): Unit = { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> mode) { + checkAnswer(output, res) + } + } + + check( + "legacy", + Seq( + Row(1, Date.valueOf("2020-01-01"), Timestamp.valueOf("2020-01-01 00:00:00")), + Row(2, Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + check( + "corrected", + Seq( + Row(1, null, null), + Row(2, Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + val err = intercept[TestFailedException] { + check("exception", Nil) + } + assert(err.message.get.contains("org.apache.spark.SparkUpgradeException")) + } + } +} + +class GlutenCSVLegacyTimeParserSuite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, "legacy") +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.scala new file mode 100644 index 000000000000..132e80696cff --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.scala @@ -0,0 +1,24 @@ +/* + * 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.spark.sql.execution.datasources.exchange + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.exchange.ValidateRequirementsSuite + +class GlutenValidateRequirementsSuite + extends ValidateRequirementsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala new file mode 100644 index 000000000000..45e8e3e9e7ba --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala @@ -0,0 +1,140 @@ +/* + * 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.spark.sql.execution.datasources.json + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{sources, GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.execution.datasources.{InMemoryFileIndex, NoopCache} +import org.apache.spark.sql.execution.datasources.v2.json.JsonScanBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DateType, IntegerType, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import org.scalatest.exceptions.TestFailedException + +import java.sql.{Date, Timestamp} + +class GlutenJsonSuite extends JsonSuite with GlutenSQLTestsBaseTrait { + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenJsonV1Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "json") +} + +class GlutenJsonV2Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + testGluten("get pushed filters") { + val attr = "col" + def getBuilder(path: String): JsonScanBuilder = { + val fileIndex = new InMemoryFileIndex( + spark, + Seq(new org.apache.hadoop.fs.Path(path, "file.json")), + Map.empty, + None, + NoopCache) + val schema = new StructType().add(attr, IntegerType) + val options = CaseInsensitiveStringMap.empty() + new JsonScanBuilder(spark, fileIndex, schema, schema, options) + } + val filters: Array[sources.Filter] = Array(sources.IsNotNull(attr)) + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === filters) + } + } + + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "false") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === Array.empty[sources.Filter]) + } + } + } + + testGluten("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") { + withTempPath { + path => + Seq( + """{"date": "2020011", "ts": "2020011"}""", + """{"date": "20201203", "ts": "20201203"}""") + .toDF() + .repartition(1) + .write + .text(path.getAbsolutePath) + val schema = new StructType() + .add("date", DateType) + .add("ts", TimestampType) + val output = spark.read + .schema(schema) + .option("dateFormat", "yyyyMMdd") + .option("timestampFormat", "yyyyMMdd") + .json(path.getAbsolutePath) + + def check(mode: String, res: Seq[Row]): Unit = { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> mode) { + checkAnswer(output, res) + } + } + + check( + "legacy", + Seq( + Row(Date.valueOf("2020-01-01"), Timestamp.valueOf("2020-01-01 00:00:00")), + Row(Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + check( + "corrected", + Seq( + Row(null, null), + Row(Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + val err = intercept[TestFailedException] { + check("exception", Nil) + } + assert(err.message.get.contains("org.apache.spark.SparkUpgradeException")) + } + } +} + +class GlutenJsonLegacyTimeParserSuite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + + override def excluded: Seq[String] = + Seq("Write timestamps correctly in ISO8601 format by default") + + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, "legacy") +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala new file mode 100644 index 000000000000..e2e3818aad9c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcColumnarBatchReaderSuite + extends OrcColumnarBatchReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala new file mode 100644 index 000000000000..f5a8db3395d6 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** A test suite that tests Apache ORC filter API based filter pushdown optimization. */ +class GlutenOrcFilterSuite extends OrcFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..a9848b7f444d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcPartitionDiscoverySuite + extends OrcPartitionDiscoveryTest + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1PartitionDiscoverySuite + extends OrcV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala new file mode 100644 index 000000000000..f186695b88b8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala @@ -0,0 +1,156 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.SQLConf + +class GlutenOrcQuerySuite extends OrcQuerySuite with GlutenSQLTestsBaseTrait { + testGluten("Simple selection form ORC table") { + val data = (1 to 10).map { + i => Person(s"name_$i", i, (0 to 1).map(m => Contact(s"contact_$m", s"phone_$m"))) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = leaf-0 + assert(sql("SELECT name FROM t WHERE age <= 5").count() === 5) + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = (not leaf-0) + assertResult(10) { + sql("SELECT name, contacts FROM t where age > 5").rdd + .flatMap(_.getAs[scala.collection.Seq[_]]("contacts")) + .count() + } + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // leaf-1 = (LESS_THAN age 8) + // expr = (and (not leaf-0) leaf-1) + { + val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8") + assert(df.count() === 2) + assertResult(4) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + + // ppd: + // leaf-0 = (LESS_THAN age 2) + // leaf-1 = (LESS_THAN_EQUALS age 8) + // expr = (or leaf-0 (not leaf-1)) + { + val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8") + assert(df.count() === 3) + assertResult(6) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + } + } + } + + testGluten("simple select queries") { + withOrcTable((0 until 10).map(i => (i, i.toString)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t where t.`_1` > 5"), (6 until 10).map(Row.apply(_))) + + checkAnswer( + sql("SELECT `_1` FROM t as tmp where tmp.`_1` < 5"), + (0 until 5).map(Row.apply(_))) + } + } + } + + testGluten("overwriting") { + val data = (0 until 10).map(i => (i, i.toString)) + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") + checkAnswer(spark.table("t"), data.map(Row.fromTuple)) + } + } + spark.sessionState.catalog.dropTable( + TableIdentifier("tmp"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("self-join") { + // 4 rows, cells of column 1 of row 2 and row 4 are null + val data = (1 to 4).map { + i => + val maybeInt = if (i % 2 == 0) None else Some(i) + (maybeInt, i.toString) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x.`_1` = y.`_1`") + val queryOutput = selfJoin.queryExecution.analyzed.output + + assertResult(4, "Field count mismatches")(queryOutput.size) + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { + queryOutput.filter(_.name == "_1").map(_.exprId).size + } + + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + } + } + } + + testGluten("columns only referenced by pushed down filters should remain") { + withOrcTable((1 to 10).map(Tuple1.apply), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t WHERE `_1` < 10"), (1 to 9).map(Row.apply(_))) + } + } + } + + testGluten("SPARK-5309 strings stored using dictionary compression in orc") { + withOrcTable((0 until 1000).map(i => ("same", "run_" + i / 100, 1)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t GROUP BY `_1`, `_2`"), + (0 until 10).map(i => Row("same", "run_" + i, 100))) + + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t WHERE `_2` = 'run_5' GROUP BY `_1`, `_2`"), + List(Row("same", "run_5", 100))) + } + } + } +} + +class GlutenOrcV1QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "orc") +} + +class GlutenOrcV2QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala new file mode 100644 index 000000000000..3413ff72c100 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala @@ -0,0 +1,183 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants, Row} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DayTimeIntervalType, IntegerType, StructField, StructType, YearMonthIntervalType} + +import java.sql.Date +import java.time.{Duration, Period} + +class GlutenOrcSourceSuite extends OrcSourceSuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + override def withAllNativeOrcReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")(code) + } + + testGluten("SPARK-31238: compatibility with Spark 2.4 in reading dates") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_date_v2_4.snappy.orc"), + Row(java.sql.Date.valueOf("1200-01-01"))) + } + } + } + + testGluten("SPARK-31238, SPARK-31423: rebasing dates in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01", "1582-10-10") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq(Row(Date.valueOf("1001-01-01")), Row(Date.valueOf("1582-10-15")))) + } + } + } + } + + testGluten("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_ts_v2_4.snappy.orc"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + + testGluten("SPARK-31284, SPARK-31423: rebasing timestamps in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01 01:02:03.123456", "1582-10-10 11:12:13.654321") + .toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq( + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456")), + Row(java.sql.Timestamp.valueOf("1582-10-15 11:12:13.654321")))) + } + } + } + } + + testGluten("SPARK-34862: Support ORC vectorized reader for nested column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val df = spark + .range(10) + .map { + x => + val stringColumn = s"$x" * 10 + val structColumn = (x, s"$x" * 100) + val arrayColumn = (0 until 5).map(i => (x + i, s"$x" * 5)) + val mapColumn = Map( + s"$x" -> (x * 0.1, (x, s"$x" * 100)), + (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)), + (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300))) + (x, stringColumn, structColumn, arrayColumn, mapColumn) + } + .toDF("int_col", "string_col", "struct_col", "array_col", "map_col") + df.write.format("orc").save(path) + + // Rewrite because Gluten does not support Spark's vectorized reading. + withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") { + val readDf = spark.read.orc(path) + val vectorizationEnabled = readDf.queryExecution.executedPlan.find { + case scan: FileSourceScanExec => scan.supportsColumnar + case _ => false + }.isDefined + assert(!vectorizationEnabled) + checkAnswer(readDf, df) + } + } + } + withAllNativeOrcReaders { + Seq(false).foreach { + vecReaderNestedColEnabled => + val vecReaderEnabled = SQLConf.get.orcVectorizedReaderEnabled + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-36931: Support reading and writing ANSI intervals (" + + s"${SQLConf.ORC_VECTORIZED_READER_ENABLED.key}=$vecReaderEnabled, " + + s"${SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key}" + + s"=$vecReaderNestedColEnabled)") { + + withSQLConf( + SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> + vecReaderEnabled.toString, + SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> + vecReaderNestedColEnabled.toString + ) { + Seq( + YearMonthIntervalType() -> ((i: Int) => Period.of(i, i, 0)), + DayTimeIntervalType() -> ((i: Int) => Duration.ofDays(i).plusSeconds(i)) + ).foreach { + case (it, f) => + val data = (1 to 10).map(i => Row(i, f(i))) + val schema = StructType( + Array(StructField("d", IntegerType, false), StructField("i", it, false))) + withTempPath { + file => + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + + // Tests for ANSI intervals in complex types. + withTempPath { + file => + val df = spark.sql("""SELECT + | named_struct('interval', interval '1-2' year to month) a, + | array(interval '1 2:3' day to minute) b, + | map('key', interval '10' year) c, + | map(interval '20' second, 'value') d""".stripMargin) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala new file mode 100644 index 000000000000..3c2fb0b318f1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcV1FilterSuite extends OrcV1FilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala new file mode 100644 index 000000000000..c142d33bdc22 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala @@ -0,0 +1,51 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.gluten.execution.FileSourceScanExecTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV1SchemaPruningSuite extends OrcV1SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: FileSourceScanExec => scan.requiredSchema + case scan: FileSourceScanExecTransformer => scan.requiredSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala new file mode 100644 index 000000000000..e47deb0d70b4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala @@ -0,0 +1,52 @@ +/* + * 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.spark.sql.execution.datasources.orc + +import org.apache.gluten.execution.BatchScanExecTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV2SchemaPruningSuite extends OrcV2SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case BatchScanExec(_, scan: OrcScan, _, _, _, _) => scan.readDataSchema + case BatchScanExecTransformer(_, scan: OrcScan, _, _, _, _, _, _, _) => scan.readDataSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala new file mode 100644 index 000000000000..60e1ca04a2d4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} + +class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait { + private val actions: Seq[DataFrame => DataFrame] = Seq( + "_1 = 500", + "_1 = 500 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1000 or _1 = 1500", + "_1 >= 500 and _1 < 1000", + "(_1 >= 500 and _1 < 1000) or (_1 >= 1500 and _1 < 1600)" + ).map(f => (df: DataFrame) => df.filter(f)) + + testGluten("test reading unaligned pages - test all types") { + val df = spark + .range(0, 2000) + .selectExpr( + "id as _1", + "cast(id as short) as _3", + "cast(id as int) as _4", + "cast(id as float) as _5", + "cast(id as double) as _6", + "cast(id as decimal(20,0)) as _7", + // We changed 1618161925000 to 1618161925 to avoid reaching the limitation of Velox: + // Timepoint is outside of supported year range. + "cast(cast(1618161925 + id * 60 * 60 * 24 as timestamp) as date) as _9" + ) + checkUnalignedPages(df)(actions: _*) + } + + testGluten("test reading unaligned pages - test all types (dict encode)") { + val df = spark + .range(0, 2000) + .selectExpr( + "id as _1", + "cast(id % 10 as byte) as _2", + "cast(id % 10 as short) as _3", + "cast(id % 10 as int) as _4", + "cast(id % 10 as float) as _5", + "cast(id % 10 as double) as _6", + "cast(id % 10 as decimal(20,0)) as _7", + "cast(id % 2 as boolean) as _8", + "cast(cast(1618161925 + (id % 10) * 60 * 60 * 24 as timestamp) as date) as _9", + "cast(1618161925 + (id % 10) as timestamp) as _10" + ) + checkUnalignedPages(df)(actions: _*) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala new file mode 100644 index 000000000000..ac938d4eaf4f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +class GlutenParquetCompressionCodecPrecedenceSuite + extends ParquetCompressionCodecPrecedenceSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..166f3255efd5 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala new file mode 100644 index 000000000000..ccb69819a3a3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaEncodingInteger + extends ParquetDeltaEncodingInteger + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetDeltaEncodingLong + extends ParquetDeltaEncodingLong + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..36928cee001d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaLengthByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala new file mode 100644 index 000000000000..6c69c700becc --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// TODO: this needs a lot more testing but it's currently not easy to test with the parquet +// writer abstractions. Revisit. +class GlutenParquetEncodingSuite extends ParquetEncodingSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala new file mode 100644 index 000000000000..bd1c269843fb --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -0,0 +1,42 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} + +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala new file mode 100644 index 000000000000..b60850df2401 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetFileFormatV1Suite + extends ParquetFileFormatV1Suite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetFileFormatV2Suite + extends ParquetFileFormatV2Suite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala new file mode 100644 index 000000000000..9d5291471b33 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetFileMetadataStructRowIndexSuite + extends ParquetFileMetadataStructRowIndexSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala new file mode 100644 index 000000000000..3a039d975f30 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -0,0 +1,580 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, LEGACY} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 +import org.apache.spark.sql.types._ +import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path +import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate, Operators} +import org.apache.parquet.filter2.predicate.FilterApi._ +import org.apache.parquet.filter2.predicate.Operators.{Column => _, Eq, Gt, GtEq, Lt, LtEq, NotEq} +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetOutputFormat} +import org.apache.parquet.hadoop.util.HadoopInputFile + +import java.sql.{Date, Timestamp} +import java.time.LocalDate + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQLTestsBaseTrait { + protected def checkFilterPredicate( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: Seq[Row])(implicit df: DataFrame): Unit = { + checkFilterPredicate(df, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected) + } + + protected def checkFilterPredicate[T]( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: T)(implicit df: DataFrame): Unit = { + checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) + } + + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) + } + + testGluten("filter pushdown - timestamp") { + Seq(true, false).foreach { + java8Api => + Seq(CORRECTED, LEGACY).foreach { + rebaseMode => + val millisData = Seq( + "1000-06-14 08:28:53.123", + "1582-06-15 08:28:53.001", + "1900-06-16 08:28:53.0", + "2018-06-17 08:28:53.999") + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString + ) { + import testImplicits._ + withTempPath { + file => + millisData + .map(i => Tuple1(Timestamp.valueOf(i))) + .toDF + .write + .format(dataSourceName) + .save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { + df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } + } + } + } + } + } + } + + testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + dir => + val path = s"${dir.getCanonicalPath}/table1" + val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") + df.show() + df.write.parquet(path) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2) or not(b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2 and b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + } + } + } + + testGluten("SPARK-23852: Broken Parquet push-down for partially-written stats") { + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + // parquet-1217.parquet contains a single column with values -1, 0, 1, 2 and null. + // The row-group statistics include null counts, but not min and max values, which + // triggers PARQUET-1217. + + val df = readResourceParquetFile("test-data/parquet-1217.parquet") + + // Will return 0 rows if PARQUET-1217 is not fixed. + assert(df.where("col > 0").count() === 2) + } + } + + testGluten("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + val schema = StructType( + Seq( + StructField("a", IntegerType, nullable = false) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + assertResult(Some(FilterApi.eq(intColumn("a"), null: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(null))) + } + + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10))) + } + + // Remove duplicates + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10, 10))) + } + + assertResult( + Some( + or( + or(FilterApi.eq(intColumn("a"), 10: Integer), FilterApi.eq(intColumn("a"), 20: Integer)), + FilterApi.eq(intColumn("a"), 30: Integer)))) { + parquetFilters.createFilter(sources.In("a", Array(10, 20, 30))) + } + + Seq(0, 10).foreach { + threshold => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> threshold.toString) { + assert( + createParquetFilters(parquetSchema) + .createFilter(sources.In("a", Array(10, 20, 30))) + .nonEmpty === threshold > 0) + } + } + + import testImplicits._ + withTempPath { + path => + val data = 0 to 1024 + data + .toDF("a") + .selectExpr("if (a = 1024, null, a) AS a") // convert 1024 to null + .coalesce(1) + .write + .option("parquet.block.size", 512) + .parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + Seq(true, false).foreach { + pushEnabled => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushEnabled.toString) { + Seq(1, 5, 10, 11, 100).foreach { + count => + val filter = s"a in(${Range(0, count).mkString(",")})" + assert(df.where(filter).count() === count) + val actual = stripSparkFilter(df.where(filter)).collect().length + assert(actual === count) + } + assert(df.where("a in(null)").count() === 0) + assert(df.where("a = null").count() === 0) + assert(df.where("a is null").count() === 1) + } + } + } + } + + // Velox doesn't support ParquetOutputFormat.PAGE_SIZE and ParquetOutputFormat.BLOCK_SIZE. + ignoreGluten("Support Parquet column index") { + // block 1: + // null count min max + // page-0 0 0 99 + // page-1 0 100 199 + // page-2 0 200 299 + // page-3 0 300 399 + // page-4 0 400 449 + // + // block 2: + // null count min max + // page-0 0 450 549 + // page-1 0 550 649 + // page-2 0 650 749 + // page-3 0 750 849 + // page-4 0 850 899 + withTempPath { + path => + spark + .range(900) + .repartition(1) + .write + .option(ParquetOutputFormat.PAGE_SIZE, "500") + .option(ParquetOutputFormat.BLOCK_SIZE, "2000") + .parquet(path.getCanonicalPath) + + val parquetFile = path.listFiles().filter(_.getName.startsWith("part")).last + val in = HadoopInputFile.fromPath( + new Path(parquetFile.getCanonicalPath), + spark.sessionState.newHadoopConf()) + + Utils.tryWithResource(ParquetFileReader.open(in)) { + reader => + val blocks = reader.getFooter.getBlocks + assert(blocks.size() > 1) + val columns = blocks.get(0).getColumns + assert(columns.size() === 1) + val columnIndex = reader.readColumnIndex(columns.get(0)) + assert(columnIndex.getMinValues.size() > 1) + + val rowGroupCnt = blocks.get(0).getRowCount + // Page count = Second page min value - first page min value + val pageCnt = columnIndex.getMinValues.get(1).asLongBuffer().get() - + columnIndex.getMinValues.get(0).asLongBuffer().get() + assert(pageCnt < rowGroupCnt) + Seq(true, false).foreach { + columnIndex => + withSQLConf(ParquetInputFormat.COLUMN_INDEX_FILTERING_ENABLED -> s"$columnIndex") { + val df = spark.read.parquet(parquetFile.getCanonicalPath).where("id = 1") + df.collect() + val plan = df.queryExecution.executedPlan + // Ignore metrics comparison. + /* + val metrics = plan.collectLeaves().head.metrics + val numOutputRows = metrics("numOutputRows").value + + if (columnIndex) { + assert(numOutputRows === pageCnt) + } else { + assert(numOutputRows === rowGroupCnt) + } + */ + } + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV1FilterSuite extends GlutenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "parquet") + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + Seq(("parquet", true), ("", false)).foreach { + case (pushdownDsList, nestedPredicatePushdown) => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST.key -> pushdownDsList + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + val nestedOrAttributes = predicate.collectFirst { + case g: GetStructField => g + case a: Attribute => a + } + assert(nestedOrAttributes.isDefined, "No GetStructField nor Attribute is detected.") + + val parsed = + parseColumnPath(PushableColumnAndNestedColumn.unapply(nestedOrAttributes.get).get) + + val containsNestedColumnOrDot = parsed.length > 1 || parsed(0).contains(".") + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan + .collect { + case PhysicalOperation( + _, + filters, + LogicalRelation(relation: HadoopFsRelation, _, _, _, _)) => + maybeRelation = Some(relation) + filters + } + .flatten + .reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + // If predicates contains nested column or dot, we push down the predicates only if + // "parquet" is in `NESTED_PREDICATE_PUSHDOWN_V1_SOURCE_LIST`. + if (nestedPredicatePushdown || !containsNestedColumnOrDot) { + assert(selectedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(selectedFilters) === selectedFilters) + val pushedParquetFilters = selectedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + } else { + assert(selectedFilters.isEmpty, "There is filter pushed down") + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV2FilterSuite extends GlutenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false" + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan.collectFirst { + case PhysicalOperation( + _, + filters, + DataSourceV2ScanRelation(_, scan: ParquetScan, _, None, None)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter(_, true)).toArray + val pushedFilters = scan.pushedFilters + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(sourceFilters) === pushedFilters) + val pushedParquetFilters = pushedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + + case _ => assert(false, "Can not match ParquetTable in the query.") + } + } + } + + /** + * Takes a sequence of products `data` to generate multi-level nested dataframes as new test data. + * It tests both non-nested and nested dataframes which are written and read back with Parquet + * datasource. + * + * This is different from [[ParquetTest.withParquetDataFrame]] which does not test nested cases. + */ + private def withNestedParquetDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T])( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = + withNestedParquetDataFrame(spark.createDataFrame(data))(runTest) + + private def withNestedParquetDataFrame(inputDF: DataFrame)( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + withNestedDataFrame(inputDF).foreach { + case (newDF, colName, resultFun) => + withTempPath { + file => + newDF.write.format(dataSourceName).save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath)(df => runTest(df, colName, resultFun)) + } + } + } + + testGluten("filter pushdown - date") { + implicit class StringToDate(s: String) { + def date: Date = Date.valueOf(s) + } + + val data = Seq("1000-01-01", "2018-03-19", "2018-03-20", "2018-03-21") + import testImplicits._ + + // Velox backend does not support rebaseMode being LEGACY. + Seq(false, true).foreach { + java8Api => + Seq(CORRECTED).foreach { + rebaseMode => + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString) { + val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() + withNestedParquetDataFrame(dates) { + case (inputDF, colName, fun) => + implicit val df: DataFrame = inputDF + + def resultFun(dateStr: String): Any = { + val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) + fun(parsed) + } + + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate( + dateAttr.isNotNull, + classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr === "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + logWarning(s"java8Api: $java8Api, rebaseMode, $rebaseMode") + checkFilterPredicate( + dateAttr <=> "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr =!= "1000-01-01".date, + classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr < "2018-03-19".date, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr > "2018-03-20".date, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr <= "1000-01-01".date, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr >= "2018-03-21".date, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + Literal("1000-01-01".date) === dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("1000-01-01".date) <=> dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-19".date) > dateAttr, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-20".date) < dateAttr, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + Literal("1000-01-01".date) >= dateAttr, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-21".date) <= dateAttr, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + !(dateAttr < "2018-03-21".date), + classOf[GtEq[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("1000-01-01")), Row(resultFun("2018-03-21")))) + + Seq(3, 20).foreach { + threshold => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> s"$threshold") { + checkFilterPredicate( + In( + dateAttr, + Array( + "2018-03-19".date, + "2018-03-20".date, + "2018-03-21".date, + "2018-03-22".date).map(Literal.apply)), + if (threshold == 3) classOf[Operators.In[_]] else classOf[Operators.Or], + Seq( + Row(resultFun("2018-03-19")), + Row(resultFun("2018-03-20")), + Row(resultFun("2018-03-21"))) + ) + } + } + } + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala new file mode 100644 index 000000000000..ad1ae40f928c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +/** A test suite that tests basic Parquet I/O. */ +class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } + + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet(testFile(name)) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala new file mode 100644 index 000000000000..051343dafb06 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetInteroperabilitySuite + extends ParquetInteroperabilitySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..5af8fa48c538 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +class GlutenParquetV1PartitionDiscoverySuite + extends ParquetV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetV2PartitionDiscoverySuite + extends ParquetV2PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala new file mode 100644 index 000000000000..f175910792bd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} + +class GlutenParquetProtobufCompatibilitySuite + extends ParquetProtobufCompatibilitySuite + with GlutenSQLTestsBaseTrait { + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala new file mode 100644 index 000000000000..07ef21208cd0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala @@ -0,0 +1,55 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +/** A test suite that tests various Parquet queries. */ +class GlutenParquetV1QuerySuite extends ParquetV1QuerySuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} + +class GlutenParquetV2QuerySuite extends ParquetV2QuerySuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten( + "SPARK-26677: negated null-safe equality comparison " + + "should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala new file mode 100644 index 000000000000..dbc2dac3df00 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.internal.LegacyBehaviorPolicy +import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, EXCEPTION, LEGACY} +import org.apache.spark.sql.internal.SQLConf + +import java.sql.Date + +class GlutenParquetRebaseDatetimeV1Suite + extends ParquetRebaseDatetimeV1Suite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } + + private def inReadConfToOptions( + conf: String, + mode: LegacyBehaviorPolicy.Value): Map[String, String] = conf match { + case SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ.key => + Map(ParquetOptions.INT96_REBASE_MODE -> mode.toString) + case _ => Map(ParquetOptions.DATETIME_REBASE_MODE -> mode.toString) + } + + private def runInMode(conf: String, modes: Seq[LegacyBehaviorPolicy.Value])( + f: Map[String, String] => Unit): Unit = { + modes.foreach(mode => withSQLConf(conf -> mode.toString)(f(Map.empty))) + withSQLConf(conf -> EXCEPTION.toString) { + modes.foreach(mode => f(inReadConfToOptions(conf, mode))) + } + } + + // gluten does not consider file metadata which indicates needs rebase or not + // it only supports write the parquet file as CORRECTED + testGluten("SPARK-31159: rebasing dates in write") { + val N = 8 + Seq(false, true).foreach { + dictionaryEncoding => + withTempPath { + dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) { + Seq + .tabulate(N)(_ => "1001-01-01") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEncoding) + .parquet(path) + } + + withAllParquetReaders { + // The file metadata indicates if it needs rebase or not, so we can always get the + // correct result regardless of the "rebase mode" config. + runInMode( + SQLConf.PARQUET_REBASE_MODE_IN_READ.key, + Seq(LEGACY, CORRECTED, EXCEPTION)) { + options => + checkAnswer( + spark.read.options(options).parquet(path), + Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-01")))) + } + + // Force to not rebase to prove the written datetime values are rebased + // and we will get wrong result if we don't rebase while reading. + // gluten not support this mode +// withSQLConf("spark.test.forceNoRebase" -> "true") { +// checkAnswer( +// spark.read.parquet(path), +// Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-07")))) +// } + } + } + } + } +} + +class GlutenParquetRebaseDatetimeV2Suite + extends ParquetRebaseDatetimeV2Suite + with GlutenSQLTestsBaseTrait { + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala new file mode 100644 index 000000000000..5cf41b7a9ed5 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala @@ -0,0 +1,361 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +import org.apache.spark.sql.functions.{col, max, min} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{LongType, StringType} + +import org.apache.hadoop.fs.Path +import org.apache.parquet.column.ParquetProperties._ +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE + +import java.io.File + +import scala.collection.JavaConverters._ + +class GlutenParquetRowIndexSuite extends ParquetRowIndexSuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + private def readRowGroupRowCounts(path: String): Seq[Long] = { + ParquetFooterReader + .readFooter( + spark.sessionState.newHadoopConf(), + new Path(path), + ParquetMetadataConverter.NO_FILTER) + .getBlocks + .asScala + .map(_.getRowCount) + .toSeq + } + + private def readRowGroupRowCounts(dir: File): Seq[Seq[Long]] = { + assert(dir.isDirectory) + dir + .listFiles() + .filter(f => f.isFile && f.getName.endsWith("parquet")) + .map(f => readRowGroupRowCounts(f.getAbsolutePath)) + .toSeq + } + + /** Do the files contain exactly one row group? */ + private def assertOneRowGroup(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => assert(rcs.length == 1, "expected one row group per file") + } + } + + /** + * Do the files have a good layout to test row group skipping (both range metadata filter, and by + * using min/max). + */ + private def assertTinyRowGroups(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => + assert(rcs.length > 1, "expected multiple row groups per file") + assert(rcs.last <= DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK) + assert( + rcs.reverse.tail.distinct == Seq(DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK), + "expected row groups with minimal row count") + } + } + + /** + * Do the files have a good layout to test a combination of page skipping and row group skipping? + */ + private def assertIntermediateRowGroups(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => + assert(rcs.length >= 3, "expected at least 3 row groups per file") + rcs.reverse.tail.foreach { + rc => + assert( + rc > DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK, + "expected row groups larger than minimal row count") + } + } + } + + case class GlutenRowIndexTestConf( + numRows: Long = 10000L, + useMultipleFiles: Boolean = false, + useVectorizedReader: Boolean = true, + useSmallPages: Boolean = false, + useSmallRowGroups: Boolean = false, + useSmallSplits: Boolean = false, + useFilter: Boolean = false, + useDataSourceV2: Boolean = false) { + + val NUM_MULTIPLE_FILES = 4 + // The test doesn't work correctly if the number of records per file is uneven. + assert(!useMultipleFiles || (numRows % NUM_MULTIPLE_FILES == 0)) + + def numFiles: Int = if (useMultipleFiles) { NUM_MULTIPLE_FILES } + else { 1 } + + def rowGroupSize: Long = if (useSmallRowGroups) { + if (useSmallPages) { + // Each file will contain multiple row groups. All of them (except for the last one) + // will contain more than DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK, so that individual + // pages within the row group can be skipped. + 2048L + } else { + // Each file will contain multiple row groups. All of them (except for the last one) + // will contain exactly DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK records. + 64L + } + } else { + // Each file will contain a single row group. + DEFAULT_BLOCK_SIZE + } + + def pageSize: Long = if (useSmallPages) { + // Each page (except for the last one for each column) will contain exactly + // DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK records. + 64L + } else { + DEFAULT_PAGE_SIZE + } + + def writeFormat: String = "parquet" + def readFormat: String = if (useDataSourceV2) { + classOf[ParquetDataSourceV2].getCanonicalName + } else { + "parquet" + } + + assert(useSmallRowGroups || !useSmallSplits) + def filesMaxPartitionBytes: Long = if (useSmallSplits) { + 256L + } else { + SQLConf.FILES_MAX_PARTITION_BYTES.defaultValue.get + } + + def desc: String = { + { if (useVectorizedReader) Seq("vectorized reader") else Seq("parquet-mr reader") } ++ { + if (useMultipleFiles) Seq("many files") else Seq.empty[String] + } ++ { if (useFilter) Seq("filtered") else Seq.empty[String] } ++ { + if (useSmallPages) Seq("small pages") else Seq.empty[String] + } ++ { if (useSmallRowGroups) Seq("small row groups") else Seq.empty[String] } ++ { + if (useSmallSplits) Seq("small splits") else Seq.empty[String] + } ++ { if (useDataSourceV2) Seq("datasource v2") else Seq.empty[String] } + }.mkString(", ") + + def sqlConfs: Seq[(String, String)] = Seq( + // TODO: remove this change after customized parquet options as `block_size`, `page_size` + // been fully supported. + GlutenConfig.NATIVE_WRITER_ENABLED.key -> "false", + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> useVectorizedReader.toString, + SQLConf.FILES_MAX_PARTITION_BYTES.key -> filesMaxPartitionBytes.toString + ) ++ { if (useDataSourceV2) Seq(SQLConf.USE_V1_SOURCE_LIST.key -> "") else Seq.empty } + } + + for (useVectorizedReader <- Seq(true, false)) + for (useDataSourceV2 <- Seq(true, false)) + for (useSmallRowGroups <- Seq(true, false)) + for (useSmallPages <- Seq(true, false)) + for (useFilter <- Seq(true, false)) + for (useSmallSplits <- Seq(useSmallRowGroups, false).distinct) { + val conf = GlutenRowIndexTestConf( + useVectorizedReader = useVectorizedReader, + useDataSourceV2 = useDataSourceV2, + useSmallRowGroups = useSmallRowGroups, + useSmallPages = useSmallPages, + useFilter = useFilter, + useSmallSplits = useSmallSplits + ) + testRowIndexGeneration("row index generation", conf) + } + + private def testRowIndexGeneration(label: String, conf: GlutenRowIndexTestConf): Unit = { + testGluten(s"$label - ${conf.desc}") { + withSQLConf(conf.sqlConfs: _*) { + withTempPath { + path => + // Read row index using _metadata.row_index if that is supported by the file format. + val rowIndexMetadataColumnSupported = conf.readFormat match { + case "parquet" => true + case _ => false + } + val rowIndexColName = if (rowIndexMetadataColumnSupported) { + s"${FileFormat.METADATA_NAME}.${ParquetFileFormat.ROW_INDEX}" + } else { + ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + } + val numRecordsPerFile = conf.numRows / conf.numFiles + val (skipCentileFirst, skipCentileMidLeft, skipCentileMidRight, skipCentileLast) = + (0.2, 0.4, 0.6, 0.8) + val expectedRowIdxCol = "expected_rowIdx_col" + val df = spark + .range(0, conf.numRows, 1, conf.numFiles) + .toDF("id") + .withColumn("dummy_col", ($"id" / 55).cast("int")) + .withColumn(expectedRowIdxCol, ($"id" % numRecordsPerFile).cast("int")) + + // Add row index to schema if required. + val schemaWithRowIdx = if (rowIndexMetadataColumnSupported) { + df.schema + } else { + df.schema.add(rowIndexColName, LongType, nullable = true) + } + + df.write + .format(conf.writeFormat) + .option(ParquetOutputFormat.BLOCK_SIZE, conf.rowGroupSize) + .option(ParquetOutputFormat.PAGE_SIZE, conf.pageSize) + .option(ParquetOutputFormat.DICTIONARY_PAGE_SIZE, conf.pageSize) + .save(path.getAbsolutePath) + val dfRead = spark.read + .format(conf.readFormat) + .schema(schemaWithRowIdx) + .load(path.getAbsolutePath) + // Verify that the produced files are laid out as expected. + if (conf.useSmallRowGroups) { + if (conf.useSmallPages) { + assertIntermediateRowGroups(path) + } else { + assertTinyRowGroups(path) + } + } else { + assertOneRowGroup(path) + } + + val dfToAssert = if (conf.useFilter) { + // Add a filter such that we skip 60% of the records: + // [0%, 20%], [40%, 60%], [80%, 100%] + dfRead.filter( + ($"id" >= (skipCentileFirst * conf.numRows).toInt && + $"id" < (skipCentileMidLeft * conf.numRows).toInt) || + ($"id" >= (skipCentileMidRight * conf.numRows).toInt && + $"id" < (skipCentileLast * conf.numRows).toInt)) + } else { + dfRead + } + + var numPartitions: Long = 0 + var numOutputRows: Long = 0 + dfToAssert.collect() + logInfo(dfToAssert.queryExecution.executedPlan.toString()) + dfToAssert.queryExecution.executedPlan.foreach { + case a: BatchScanExec => + numPartitions += a.inputRDD.partitions.length + numOutputRows += a.metrics("numOutputRows").value + case b: FileSourceScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value + case c: BatchScanExecTransformer => + numPartitions += c.inputRDD.partitions.length + numOutputRows += c.metrics("numOutputRows").value + case f: FileSourceScanExecTransformer => + numPartitions += f.inputRDD.partitions.length + numOutputRows += f.metrics("numOutputRows").value + case _ => + } + assert(numPartitions > 0) + assert(numOutputRows > 0) + + if (conf.useSmallSplits) { + assert(numPartitions >= 2 * conf.numFiles) + } + + // Assert that every rowIdx value matches the value in `expectedRowIdx`. + assert( + dfToAssert + .filter(s"$rowIndexColName != $expectedRowIdxCol") + .count() == 0) + + if (conf.useFilter) { + if (conf.useSmallRowGroups) { + assert(numOutputRows < conf.numRows) + } + + val minMaxRowIndexes = + dfToAssert.select(max(col(rowIndexColName)), min(col(rowIndexColName))).collect() + val (expectedMaxRowIdx, expectedMinRowIdx) = if (conf.numFiles == 1) { + // When there is a single file, we still have row group skipping, + // but that should not affect the produced rowIdx. + (conf.numRows * skipCentileLast - 1, conf.numRows * skipCentileFirst) + } else { + // For simplicity, the chosen filter skips the whole files. + // Thus all unskipped files will have the same max and min rowIdx values. + (numRecordsPerFile - 1, 0) + } + assert(minMaxRowIndexes(0).get(0) == expectedMaxRowIdx) + assert(minMaxRowIndexes(0).get(1) == expectedMinRowIdx) + if (!conf.useMultipleFiles) { + val skippedValues = List.range(0, (skipCentileFirst * conf.numRows).toInt) ++ + List.range( + (skipCentileMidLeft * conf.numRows).toInt, + (skipCentileMidRight * conf.numRows).toInt) ++ + List.range((skipCentileLast * conf.numRows).toInt, conf.numRows) + // rowIdx column should not have any of the `skippedValues`. + assert( + dfToAssert + .filter(col(rowIndexColName).isin(skippedValues: _*)) + .count() == 0) + } + } else { + // assert(numOutputRows == conf.numRows) + // When there is no filter, the rowIdx values should be in range + // [0-`numRecordsPerFile`]. + val expectedRowIdxValues = List.range(0, numRecordsPerFile) + assert( + dfToAssert + .filter(col(rowIndexColName).isin(expectedRowIdxValues: _*)) + .count() == conf.numRows) + } + } + } + } + } + for (useDataSourceV2 <- Seq(true, false)) { + val conf = GlutenRowIndexTestConf(useDataSourceV2 = useDataSourceV2) + + testGluten(s"invalid row index column type - ${conf.desc}") { + withSQLConf(conf.sqlConfs: _*) { + withTempPath { + path => + val df = spark.range(0, 10, 1, 1).toDF("id") + val schemaWithRowIdx = df.schema + .add(ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME, StringType) + + df.write + .format(conf.writeFormat) + .save(path.getAbsolutePath) + + val dfRead = spark.read + .format(conf.readFormat) + .schema(schemaWithRowIdx) + .load(path.getAbsolutePath) + + val exception = intercept[Exception](dfRead.collect()) + assert(exception.getMessage.contains(ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME)) + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala new file mode 100644 index 000000000000..854961e48f0a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala @@ -0,0 +1,88 @@ +/* + * 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.spark.sql.execution.datasources.parquet +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenParquetV1SchemaPruningSuite + extends ParquetV1SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: FileSourceScanExec => scan.requiredSchema + case scan: FileSourceScanExecTransformer => scan.requiredSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} + +@ExtendedSQLTest +class GlutenParquetV2SchemaPruningSuite + extends ParquetV2SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: BatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema + case scan: BatchScanExecTransformer => scan.scan.asInstanceOf[ParquetScan].readDataSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala new file mode 100644 index 000000000000..9dde5bce27af --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetSchemaInferenceSuite + extends ParquetSchemaInferenceSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetSchemaSuite extends ParquetSchemaSuite with GlutenSQLTestsBaseTrait { + + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala new file mode 100644 index 000000000000..6dc41c090244 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala @@ -0,0 +1,81 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} + +class GlutenParquetThriftCompatibilitySuite + extends ParquetThriftCompatibilitySuite + with GlutenSQLTestsBaseTrait { + + private val parquetFilePath = + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + + "/test-data/parquet-thrift-compat.snappy.parquet" + + testGluten("Read Parquet file generated by parquet-thrift") { + logInfo(s"""Schema of the Parquet file written by parquet-thrift: + |${readParquetSchema(parquetFilePath.toString)} + """.stripMargin) + + checkAnswer( + spark.read.parquet(parquetFilePath.toString), + (0 until 10).map { + i => + val suits = Array("SPADES", "HEARTS", "DIAMONDS", "CLUBS") + + val nonNullablePrimitiveValues = Seq( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + // Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always + // treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume + // Thrift `STRING`s are encoded using UTF-8. + s"val_$i", + s"val_$i", + // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings + suits(i % 4) + ) + + val nullablePrimitiveValues = if (i % 3 == 0) { + Seq.fill(nonNullablePrimitiveValues.length)(null) + } else { + nonNullablePrimitiveValues + } + + val complexValues = Seq( + Seq.tabulate(3)(n => s"arr_${i + n}"), + // Thrift `SET`s are converted to Parquet `LIST`s + Seq(i), + Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap, + Seq + .tabulate(3) { + n => + (i + n) -> Seq.tabulate(3) { + m => Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + } + .toMap + ) + + Row(nonNullablePrimitiveValues ++ nullablePrimitiveValues ++ complexValues: _*) + } + ) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala new file mode 100644 index 000000000000..a0cf738e52a6 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetVectorizedSuite extends ParquetVectorizedSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala new file mode 100644 index 000000000000..845af160c74d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala @@ -0,0 +1,290 @@ +/* + * 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.spark.sql.execution.datasources.text + +import org.apache.spark.{SparkConf, SparkIllegalArgumentException, TestUtils} +import org.apache.spark.sql.{AnalysisException, DataFrame, GlutenSQLTestsBaseTrait, QueryTest, Row, SaveMode} +import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Utils + +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + +abstract class GlutenTextSuite + extends QueryTest + with SharedSparkSession + with CommonFileDataSourceSuite + with GlutenSQLTestsBaseTrait { + import testImplicits._ + + override protected def dataSourceFormat = "text" + + testGluten("reading text file") { + verifyFrame(spark.read.format("text").load(testFile)) + } + + testGluten("SQLContext.read.text() API") { + verifyFrame(spark.read.text(testFile)) + } + + testGluten("SPARK-12562 verify write.text() can handle column name beyond `value`") { + val df = spark.read.text(testFile).withColumnRenamed("value", "adwrasdf") + + val tempFile = Utils.createTempDir() + tempFile.delete() + df.write.text(tempFile.getCanonicalPath) + verifyFrame(spark.read.text(tempFile.getCanonicalPath)) + + Utils.deleteRecursively(tempFile) + } + + testGluten("error handling for invalid schema") { + val tempFile = Utils.createTempDir() + tempFile.delete() + + val df = spark.range(2) + intercept[AnalysisException] { + df.write.text(tempFile.getCanonicalPath) + } + + intercept[AnalysisException] { + spark.range(2).select(df("id"), df("id") + 1).write.text(tempFile.getCanonicalPath) + } + } + + testGluten("reading partitioned data using read.textFile()") { + val ds = spark.read.textFile(textPartitioned) + val data = ds.collect() + + assert(ds.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + testGluten("support for partitioned reading using read.text()") { + val df = spark.read.text(textPartitioned) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + + testGluten("SPARK-13503 Support to specify the option for compression codec for TEXT") { + val testDf = spark.read.text(testFile) + val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") + extensionNameMap.foreach { + case (codecName, extension) => + val tempDir = Utils.createTempDir() + val tempDirPath = tempDir.getAbsolutePath + testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) + verifyFrame(spark.read.text(tempDirPath)) + } + + withTempDir { + dir => + checkError( + exception = intercept[SparkIllegalArgumentException] { + testDf.write + .option("compression", "illegal") + .mode(SaveMode.Overwrite) + .text(dir.getAbsolutePath) + }, + condition = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", + parameters = Map( + "codecName" -> "illegal", + "availableCodecs" -> "bzip2, deflate, uncompressed, snappy, none, lz4, gzip") + ) + } + } + + testGluten("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map[String, String]( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.map.output.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("compression", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + testGluten("case insensitive option") { + val extraOptions = Map[String, String]( + "mApReDuCe.output.fileoutputformat.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mApReDuCe.map.output.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mApReDuCe.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("CoMpReSsIoN", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + testGluten("SPARK-14343: select partitioning column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val ds1 = spark.range(1).selectExpr("CONCAT('val_', id)") + ds1.write.text(s"$path/part=a") + ds1.write.text(s"$path/part=b") + + checkAnswer( + spark.read.format("text").load(path).select($"part"), + Row("a") :: Row("b") :: Nil) + } + } + + testGluten("SPARK-15654: should not split gz files") { + withTempDir { + dir => + val path = dir.getCanonicalPath + val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s") + df1.write.option("compression", "gzip").mode("overwrite").text(path) + + val expected = df1.collect() + Seq(10, 100, 1000).foreach { + bytes => + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { + val df2 = spark.read.format("text").load(path) + checkAnswer(df2, expected) + } + } + } + } + + def testLineSeparator(lineSep: String): Unit = { + test(s"SPARK-23577: Support line separator - lineSep: '$lineSep'") { + // Read + val values = Seq("a", "b", "\nc") + val data = values.mkString(lineSep) + val dataWithTrailingLineSep = s"$data$lineSep" + Seq(data, dataWithTrailingLineSep).foreach { + lines => + withTempPath { + path => + Files.write(path.toPath, lines.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, Seq("a", "b", "\nc").toDF()) + } + } + + // Write + withTempPath { + path => + values.toDF().coalesce(1).write.option("lineSep", lineSep).text(path.getAbsolutePath) + val partFile = + TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) + assert(readBack === s"a${lineSep}b$lineSep\nc$lineSep") + } + + // Roundtrip + withTempPath { + path => + val df = values.toDF() + df.write.option("lineSep", lineSep).text(path.getAbsolutePath) + val readBack = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + Seq("|", "^", "::", "!!!@3", 0x1e.toChar.toString, "아").foreach { + lineSep => testLineSeparator(lineSep) + } + // scalastyle:on nonascii + + // Rewrite for file locating. + private def testFile: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-suite.txt" + } + + // Added for file locating. + private def textPartitioned: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-partitioned" + } + + /** Verifies data and schema. */ + private def verifyFrame(df: DataFrame): Unit = { + // schema + assert(df.schema == new StructType().add("value", StringType)) + + // verify content + val data = df.collect() + assert(data(0) == Row("This is a test file for the text data source")) + assert(data(1) == Row("1+1")) + // scalastyle:off nonascii + assert(data(2) == Row("数据砖头")) + // scalastyle:on nonascii + assert(data(3) == Row("\"doh\"")) + assert(data.length == 4) + } +} + +class GlutenTextV1Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "text") +} + +class GlutenTextV2Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala new file mode 100644 index 000000000000..f6d7db3849e9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2StrategySuite + extends DataSourceV2StrategySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala new file mode 100644 index 000000000000..bc6fcc3c0e9b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileTableSuite extends FileTableSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala new file mode 100644 index 000000000000..e2d8186f6874 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenV2PredicateSuite extends V2PredicateSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala new file mode 100644 index 000000000000..1409b598ac63 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.sql.execution.exchange + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenEnsureRequirementsSuite extends EnsureRequirementsSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala new file mode 100644 index 000000000000..8062596bdf6e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -0,0 +1,78 @@ +/* + * 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.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.internal.SQLConf + +/** + * This test needs setting for spark test home (its source code), e.g., appending the following + * setting for `mvn test`: -DargLine="-Dspark.test.home=/home/sparkuser/spark/". + * + * In addition, you also need build spark source code before running this test, e.g., with + * `./build/mvn -DskipTests clean package`. + */ +class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommonTrait { + + /** + * Create a new [[SparkSession]] running in local-cluster mode with unsafe and codegen enabled. + */ + override def beforeAll(): Unit = { + super.beforeAll() + val sparkBuilder = SparkSession + .builder() + .master("local-cluster[2,1,1024]") + .appName("Gluten-UT") + .master(s"local[2]") + .config(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "org.apache.gluten.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .config("spark.sql.warehouse.dir", warehouse) + // Avoid static evaluation for literal input by spark catalyst. + .config( + "spark.sql.optimizer.excludedRules", + ConstantFolding.ruleName + "," + + NullPropagation.ruleName) + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala new file mode 100644 index 000000000000..309af61a43ae --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.execution.joins + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExistenceJoinSuite extends ExistenceJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala new file mode 100644 index 000000000000..94d5629107df --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenInnerJoinSuiteForceShjOn extends InnerJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + } +} + +class GlutenInnerJoinSuiteForceShjOff extends InnerJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala new file mode 100644 index 000000000000..04a555cfce30 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOuterJoinSuiteForceShjOn extends OuterJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + } +} + +class GlutenOuterJoinSuiteForceShjOff extends OuterJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala new file mode 100644 index 000000000000..d364262be588 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.spark.sql.execution.python + +import org.apache.gluten.execution.{ColumnarToRowExecBase, FilterExecTransformer, RowToColumnarExecBase, WholeStageTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} +import org.apache.spark.sql.execution.{ColumnarInputAdapter, InputIteratorTransformer} + +class GlutenBatchEvalPythonExecSuite extends BatchEvalPythonExecSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + testGluten("Python UDF: push down deterministic FilterExecTransformer predicates") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(b) and dummyPythonUDF(a) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: AttributeReference), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(FilterExecTransformer(_: In, _), _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten("Nested Python UDF: push down deterministic FilterExecTransformer predicates") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(a, dummyPythonUDF(a, b)) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + _: AttributeReference, + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(FilterExecTransformer(_: In, _), _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten("Python UDF: no push down on non-deterministic") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("b > 4 and dummyPythonUDF(a) and rand() > 0.3") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: GreaterThan), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(_: FilterExecTransformer, _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten( + "Python UDF: push down on deterministic predicates after the first non-deterministic") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(a) and rand() > 0.3 and b > 4") + + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: GreaterThan), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(_: FilterExecTransformer, _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala new file mode 100644 index 000000000000..1cd34bbf7852 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala @@ -0,0 +1,167 @@ +/* + * 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.spark.sql.execution.python + +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +class GlutenExtractPythonUDFsSuite extends ExtractPythonUDFsSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + def collectBatchExec(plan: SparkPlan): Seq[BatchEvalPythonExec] = plan.collect { + case b: BatchEvalPythonExec => b + } + + def collectColumnarArrowExec(plan: SparkPlan): Seq[EvalPythonExec] = plan.collect { + // To check for ColumnarArrowEvalPythonExec + case b: EvalPythonExec + if !b.isInstanceOf[ArrowEvalPythonExec] && !b.isInstanceOf[BatchEvalPythonExec] => + b + } + + testGluten("Chained Scalar Pandas UDFs should be combined to a single physical node") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c", scalarPandasUDF(col("a"))) + .withColumn("d", scalarPandasUDF(col("c"))) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(arrowEvalNodes.size == 1) + } + + testGluten("Mixed Batched Python UDFs and Pandas UDF should be separate physical node") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c", batchedPythonUDF(col("a"))) + .withColumn("d", scalarPandasUDF(col("b"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 1) + assert(arrowEvalNodes.size == 1) + } + + testGluten( + "Independent Batched Python UDFs and Scalar Pandas UDFs should be combined separately") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c1", batchedPythonUDF(col("a"))) + .withColumn("c2", batchedPythonUDF(col("c1"))) + .withColumn("d1", scalarPandasUDF(col("a"))) + .withColumn("d2", scalarPandasUDF(col("d1"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 1) + assert(arrowEvalNodes.size == 1) + } + + testGluten("Dependent Batched Python UDFs and Scalar Pandas UDFs should not be combined") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c1", batchedPythonUDF(col("a"))) + .withColumn("d1", scalarPandasUDF(col("c1"))) + .withColumn("c2", batchedPythonUDF(col("d1"))) + .withColumn("d2", scalarPandasUDF(col("c2"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 2) + assert(arrowEvalNodes.size == 2) + } + + testGluten("Python UDF should not break column pruning/filter pushdown -- Parquet V2") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { + f => + spark.range(10).select($"id".as("a"), $"id".as("b")).write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExecTransformer => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExecTransformer => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 + val filters = scanNodes.head.scan.asInstanceOf[ParquetScan].pushedFilters + assert(filters.length == 2) + assert(filters.flatMap(_.references).distinct === Array("a")) + } + } + } + } + + testGluten("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { + f => + spark.range(10).select($"id".as("a"), $"id".as("b")).write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 + assert(scanNodes.head.dataFilters.length == 2) + assert( + scanNodes.head.dataFilters.flatMap(_.references.map(_.name)).distinct == Seq("a")) + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala new file mode 100644 index 000000000000..2ee1573ea07a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala @@ -0,0 +1,44 @@ +/* + * 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.spark.sql.extension + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} + +case class CustomerColumnarPreRules(session: SparkSession) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case fileSourceScan: FileSourceScanExec => + val transformer = new TestFileSourceScanExecTransformer( + fileSourceScan.relation, + fileSourceScan.output, + fileSourceScan.requiredSchema, + fileSourceScan.partitionFilters, + fileSourceScan.optionalBucketSet, + fileSourceScan.optionalNumCoalescedBuckets, + fileSourceScan.dataFilters, + fileSourceScan.tableIdentifier, + fileSourceScan.disableBucketedScan + ) + if (transformer.doValidate().ok()) { + transformer + } else { + plan + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala new file mode 100644 index 000000000000..49e4187babf4 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala @@ -0,0 +1,126 @@ +/* + * 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.spark.sql.extension + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.SparkPlan + +class GlutenCollapseProjectExecTransformerSuite extends GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("Support ProjectExecTransformer collapse") { + val query = + """ + |SELECT + | o_orderpriority + |FROM + | orders + |WHERE + | o_shippriority >= 0 + | AND EXISTS ( + | SELECT + | * + | FROM + | lineitem + | WHERE + | l_orderkey = o_orderkey + | AND l_linenumber < 10 + | ) + |ORDER BY + | o_orderpriority + |LIMIT + | 100; + |""".stripMargin + + val ordersData = Seq[(Int, Int, String)]( + (30340, 1, "3-MEDIUM"), + (31140, 1, "1-URGENT"), + (31940, 1, "2-HIGH"), + (32740, 1, "3-MEDIUM"), + (33540, 1, "5-LOW"), + (34340, 1, "2-HIGH"), + (35140, 1, "3-MEDIUM"), + (35940, 1, "1-URGENT"), + (36740, 1, "3-MEDIUM"), + (37540, 1, "4-NOT SPECIFIED") + ) + val lineitemData = Seq[(Int, Int, String)]( + (30340, 1, "F"), + (31140, 4, "F"), + (31940, 7, "O"), + (32740, 6, "O"), + (33540, 2, "F"), + (34340, 3, "F"), + (35140, 1, "O"), + (35940, 2, "F"), + (36740, 3, "F"), + (37540, 5, "O") + ) + withTable("orders", "lineitem") { + ordersData + .toDF("o_orderkey", "o_shippriority", "o_orderpriority") + .write + .format("parquet") + .saveAsTable("orders") + lineitemData + .toDF("l_orderkey", "l_linenumber", "l_linestatus") + .write + .format("parquet") + .saveAsTable("lineitem") + Seq(true, false).foreach { + collapsed => + withSQLConf( + GlutenConfig.ENABLE_COLUMNAR_PROJECT_COLLAPSE.key -> collapsed.toString, + "spark.sql.autoBroadcastJoinThreshold" -> "-1") { + val df = sql(query) + checkAnswer( + df, + Seq( + Row("1-URGENT"), + Row("1-URGENT"), + Row("2-HIGH"), + Row("2-HIGH"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("4-NOT SPECIFIED"), + Row("5-LOW") + ) + ) + assert( + getExecutedPlan(df).exists { + case _ @ProjectExecTransformer(_, _: ProjectExecTransformer) => true + case _ => false + } == !collapsed + ) + if (collapsed) { + val projectPlan = getExecutedPlan(df).collect { + case plan: ProjectExecTransformer => plan + }.head + assert(projectPlan.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isDefined) + } + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala new file mode 100644 index 000000000000..6dc265bc4fd0 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -0,0 +1,51 @@ +/* + * 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.spark.sql.extension + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsTrait + +class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.adaptive.enabled", "false") + .set( + GlutenConfig.EXTENDED_COLUMNAR_TRANSFORM_RULES.key, + "org.apache.spark.sql" + + ".extension.CustomerColumnarPreRules") + .set(GlutenConfig.EXTENDED_COLUMNAR_POST_RULES.key, "") + } + + testGluten("test customer column rules") { + withSQLConf((GlutenConfig.GLUTEN_ENABLED.key, "false")) { + sql("create table my_parquet(id int) using parquet") + sql("insert into my_parquet values (1)") + sql("insert into my_parquet values (2)") + } + withSQLConf((GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false")) { + val df = sql("select * from my_parquet") + val testFileSourceScanExecTransformer = df.queryExecution.executedPlan.collect { + case f: TestFileSourceScanExecTransformer => f + } + assert(testFileSourceScanExecTransformer.nonEmpty) + assert(testFileSourceScanExecTransformer.head.nodeNamePrefix.equals("TestFile")) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala new file mode 100644 index 000000000000..4924ee4c4f56 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.spark.sql.extension + +import org.apache.gluten.extension.injector.InjectorControl +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS + +class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) + } + + testGluten("test gluten extensions") { + assert( + spark.sessionState.columnarRules + .exists(_.isInstanceOf[InjectorControl.DisablerAware])) + + assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark))) + assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) + assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) + if (BackendTestUtils.isCHBackendLoaded()) { + assert(spark.sessionState.sqlParser.isInstanceOf[InjectorControl.DisablerAware]) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } + assert( + spark.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1) + .isDefined) + assert( + spark.sessionState.columnarRules.contains( + MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala new file mode 100644 index 000000000000..22e992955982 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -0,0 +1,62 @@ +/* + * 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.spark.sql.extension + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.FileSourceScanExecTransformerBase + +import org.apache.spark.Partition +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet + +/** Test for customer column rules */ +case class TestFileSourceScanExecTransformer( + @transient override val relation: HadoopFsRelation, + override val output: Seq[Attribute], + override val requiredSchema: StructType, + override val partitionFilters: Seq[Expression], + override val optionalBucketSet: Option[BitSet], + override val optionalNumCoalescedBuckets: Option[Int], + override val dataFilters: Seq[Expression], + override val tableIdentifier: Option[TableIdentifier], + override val disableBucketedScan: Boolean = false) + extends FileSourceScanExecTransformerBase( + relation, + output, + requiredSchema, + partitionFilters, + optionalBucketSet, + optionalNumCoalescedBuckets, + dataFilters, + tableIdentifier, + disableBucketedScan) { + override def getPartitions: Seq[Partition] = + BackendsApiManager.getTransformerApiInstance.genPartitionSeq( + relation, + requiredSchema, + getPartitionArray(), + output, + bucketedScan, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan) + + override val nodeNamePrefix: String = "TestFile" +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala new file mode 100644 index 000000000000..8edcef1c08c8 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -0,0 +1,235 @@ +/* + * 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.spark.sql.gluten + +import org.apache.gluten.GlutenBuildInfo +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.events.GlutenPlanFallbackEvent +import org.apache.gluten.execution.FileSourceScanExecTransformer +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.execution.ProjectExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.ui.{GlutenSQLAppStatusStore, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ElementTrackingStore + +import scala.collection.mutable.ArrayBuffer + +class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_UI_ENABLED.key, "true") + // The gluten ui event test suite expects the spark ui to be enable + .set(UI_ENABLED, true) + } + + testGluten("test fallback logging") { + val testAppender = new LogAppender("fallback reason") + withLogAppender(testAppender) { + withSQLConf( + GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false", + GlutenConfig.VALIDATION_LOG_LEVEL.key -> "error") { + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + sql("SELECT * FROM t").collect() + } + } + val msgRegex = + """Validation failed for plan: Scan parquet spark_catalog.default\.t\[QueryId=[0-9]+\],""" + + """ due to: \[FallbackByUserOptions\] Validation failed on node Scan parquet""" + + """ spark_catalog\.default\.t""".stripMargin + assert(testAppender.loggingEvents.exists(_.getMessage.getFormattedMessage.matches(msgRegex))) + } + } + + testGluten("test fallback event") { + val kvStore = spark.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + val glutenStore = new GlutenSQLAppStatusStore(kvStore) + assert( + glutenStore + .buildInfo() + .info + .find(_._1 == "Gluten Version") + .exists(_._2 == GlutenBuildInfo.VERSION)) + + def runExecution(sqlString: String): Long = { + var id = 0L + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerSQLExecutionStart => id = e.executionId + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + try { + sql(sqlString).collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + } finally { + spark.sparkContext.removeSparkListener(listener) + } + id + } + + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 1) + assert(execution.get.numFallbackNodes == 0) + assert(execution.get.fallbackNodeToReason.isEmpty) + + withSQLConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false") { + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 0) + assert(execution.get.numFallbackNodes == 1) + val fallbackReason = execution.get.fallbackNodeToReason.head + assert(fallbackReason._1.contains("Scan parquet spark_catalog.default.t")) + assert(fallbackReason._2.contains( + "[FallbackByUserOptions] Validation failed on node Scan parquet spark_catalog.default.t")) + } + } + + withTable("t1", "t2") { + spark.range(10).write.format("parquet").saveAsTable("t1") + spark.range(10).write.format("parquet").saveAsTable("t2") + + val id = runExecution("SELECT * FROM t1 FULL OUTER JOIN t2") + val execution = glutenStore.execution(id) + if (BackendTestUtils.isVeloxBackendLoaded()) { + assert(execution.get.numFallbackNodes == 1) + assert( + execution.get.fallbackNodeToReason.head._2 + .contains("FullOuter join is not supported with BroadcastNestedLoopJoin")) + } else { + assert(execution.get.numFallbackNodes == 2) + } + } + } + + testGluten("Improve merge fallback reason") { + spark.sql("create table t using parquet as select 1 as c1, timestamp '2023-01-01' as c2") + withTable("t") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { + try { + val df = + spark.sql("select c1, count(*) from t where c2 > timestamp '2022-01-01' group by c1") + checkAnswer(df, Row(1, 1)) + spark.sparkContext.listenerBus.waitUntilEmpty() + + // avoid failing when we support transform timestamp filter in future + val isFallback = find(df.queryExecution.executedPlan) { + _.isInstanceOf[FileSourceScanExecTransformer] + }.isEmpty + if (isFallback) { + events.exists( + _.fallbackNodeToReason.values.exists( + _.contains("Subfield filters creation not supported for input type 'TIMESTAMP'"))) + events.exists( + _.fallbackNodeToReason.values.exists( + _.contains("Timestamp is not fully supported in Filter"))) + } + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + } + + test("Add logical link to rewritten spark plan") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(GlutenConfig.EXPRESSION_BLACK_LIST.key -> "add") { + try { + val df = spark.sql("select sum(id + 1) from range(10)") + df.collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + val project = find(df.queryExecution.executedPlan) { + _.isInstanceOf[ProjectExec] + } + assert(project.isDefined) + assert( + events.exists(_.fallbackNodeToReason.values.toSet + .exists(_.contains("Not supported to map spark function name")))) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + + test("ExpandFallbackPolicy should propagate fallback reason to vanilla SparkPlan") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + spark.range(10).selectExpr("id as c1", "id as c2").write.format("parquet").saveAsTable("t") + withTable("t") { + withSQLConf( + GlutenConfig.EXPRESSION_BLACK_LIST.key -> "max", + GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { + try { + val df = spark.sql("select c2, max(c1) as id from t group by c2") + df.collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + val agg = collect(df.queryExecution.executedPlan) { case a: HashAggregateExec => a } + assert(agg.size == 2) + assert( + events.count( + _.fallbackNodeToReason.values.toSet.exists(_.contains( + "Could not find a valid substrait mapping name for max" + ))) == 2) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala new file mode 100644 index 000000000000..264a7e7836fd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -0,0 +1,123 @@ +/* + * 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.spark.sql.hive.execution + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier + +class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { + + override def sparkConf: SparkConf = { + defaultSparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, "true") + .set("spark.sql.storeAssignmentPolicy", "legacy") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + } + + testGluten("5182: Fix failed to parse post join filters") { + withSQLConf( + "spark.sql.hive.convertMetastoreParquet" -> "false", + "spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + sql("DROP TABLE IF EXISTS test_5182_0;") + sql("DROP TABLE IF EXISTS test_5182_1;") + sql( + "CREATE TABLE test_5182_0 (from_uid STRING, vgift_typeid int, vm_count int, " + + "status bigint, ts bigint, vm_typeid int) " + + "USING hive OPTIONS(fileFormat 'parquet') PARTITIONED BY (`day` STRING);") + sql( + "CREATE TABLE test_5182_1 (typeid int, groupid int, ss_id bigint, " + + "ss_start_time bigint, ss_end_time bigint) " + + "USING hive OPTIONS(fileFormat 'parquet');") + sql( + "INSERT INTO test_5182_0 partition(day='2024-03-31') " + + "VALUES('uid_1', 2, 10, 1, 11111111111, 2);") + sql("INSERT INTO test_5182_1 VALUES(2, 1, 1, 1000000000, 2111111111);") + val df = spark.sql( + "select ee.from_uid as uid,day, vgift_typeid, money from " + + "(select t_a.day, if(cast(substr(t_a.ts,1,10) as bigint) between " + + "t_b.ss_start_time and t_b.ss_end_time, t_b.ss_id, 0) ss_id, " + + "t_a.vgift_typeid, t_a.from_uid, vm_count money from " + + "(select from_uid,day,vgift_typeid,vm_count,ts from test_5182_0 " + + "where day between '2024-03-30' and '2024-03-31' and status=1 and vm_typeid=2) t_a " + + "left join test_5182_1 t_b on t_a.vgift_typeid=t_b.typeid " + + "where t_b.groupid in (1,2)) ee where ss_id=1;") + checkAnswer(df, Seq(Row("uid_1", "2024-03-31", 2, 10))) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5182_0"), + ignoreIfNotExists = true, + purge = false) + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5182_1"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("5249: Reading csv may throw Unexpected empty column") { + withSQLConf( + "spark.gluten.sql.complexType.scan.fallback.enabled" -> "false" + ) { + sql("DROP TABLE IF EXISTS test_5249;") + sql( + "CREATE TABLE test_5249 (name STRING, uid STRING) " + + "ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' " + + "STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' " + + "OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';") + sql("INSERT INTO test_5249 VALUES('name_1', 'id_1');") + val df = spark.sql( + "SELECT name, uid, count(distinct uid) total_uid_num from test_5249 " + + "group by name, uid with cube;") + checkAnswer( + df, + Seq( + Row("name_1", "id_1", 1), + Row("name_1", null, 1), + Row(null, "id_1", 1), + Row(null, null, 1))) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5249"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("GLUTEN-7116: Support outer explode") { + sql("create table if not exists test_7116 (id int, name string)") + sql("insert into test_7116 values (1, 'a,b'), (2, null), (null, 'c,d'), (3, '')") + val query = + """ + |select id, col_name + |from test_7116 lateral view outer explode(split(name, ',')) as col_name + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq(Row(1, "a"), Row(1, "b"), Row(2, null), Row(null, "c"), Row(null, "d"), Row(3, ""))) + spark.sessionState.catalog.dropTable( + TableIdentifier("test_7116"), + ignoreIfNotExists = true, + purge = false) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala new file mode 100644 index 000000000000..c93aa6640d77 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala @@ -0,0 +1,50 @@ +/* + * 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.spark.sql.hive.execution + +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.HiveTableScanExecTransformer + +class GlutenHiveSQLQuerySuite extends GlutenHiveSQLQuerySuiteBase { + + override def sparkConf: SparkConf = { + defaultSparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + } + + testGluten("hive orc scan") { + withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "false") { + sql("DROP TABLE IF EXISTS test_orc") + sql( + "CREATE TABLE test_orc (name STRING, favorite_color STRING)" + + " USING hive OPTIONS(fileFormat 'orc')") + sql("INSERT INTO test_orc VALUES('test_1', 'red')"); + val df = spark.sql("select * from test_orc") + checkAnswer(df, Seq(Row("test_1", "red"))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_orc"), + ignoreIfNotExists = true, + purge = false) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala new file mode 100644 index 000000000000..3382567ee286 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala @@ -0,0 +1,98 @@ +/* + * 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.spark.sql.hive.execution + +import org.apache.gluten.execution.TransformSupport + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsTrait} +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.internal.SQLConf + +import scala.reflect.ClassTag + +abstract class GlutenHiveSQLQuerySuiteBase extends GlutenSQLTestsTrait { + private var _spark: SparkSession = null + + override def beforeAll(): Unit = { + prepareWorkDir() + if (_spark == null) { + _spark = SparkSession.builder().config(sparkConf).enableHiveSupport().getOrCreate() + } + + _spark.sparkContext.setLogLevel("warn") + } + + override protected def spark: SparkSession = _spark + + override def afterAll(): Unit = { + try { + super.afterAll() + if (_spark != null) { + try { + _spark.sessionState.catalog.reset() + } finally { + _spark.stop() + _spark = null + } + } + } finally { + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + doThreadPostAudit() + } + } + + protected def defaultSparkConf: SparkConf = { + val conf = new SparkConf() + .set("spark.master", "local[1]") + .set("spark.sql.test", "") + .set("spark.sql.testkey", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) + .set( + HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, + "org.apache.spark.sql.hive.execution.PairSerDe") + // SPARK-8910 + .set(UI_ENABLED, false) + .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) + // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes + // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. + .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") + // Disable ConvertToLocalRelation for better test coverage. Test cases built on + // LocalRelation will exercise the optimization rules better by disabling it as + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + + conf.set( + "spark.sql.warehouse.dir", + getClass.getResource("/").getPath + "/tests-working-home/spark-warehouse") + val metastore = getClass.getResource("/").getPath + getClass.getCanonicalName + "/metastore_db" + conf.set("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastore;create=true") + } + + def checkOperatorMatch[T <: TransformSupport](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass)) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala new file mode 100644 index 000000000000..9a9f06e02c5d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenBucketedReadWithoutHiveSupportSuite + extends BucketedReadWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala new file mode 100644 index 000000000000..e5dd2de8b8bd --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.sources +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketedWriteWithoutHiveSupportSuite + extends BucketedWriteWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala new file mode 100644 index 000000000000..7f31d62f74be --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenCreateTableAsSelectSuite + extends CreateTableAsSelectSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala new file mode 100644 index 000000000000..03775cab3914 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// please note that the META-INF/services had to be modified for the test directory for this to work +class GlutenDDLSourceLoadSuite extends DDLSourceLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala new file mode 100644 index 000000000000..fd77663985bc --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala new file mode 100644 index 000000000000..84ba336099a1 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExternalCommandRunnerSuite + extends ExternalCommandRunnerSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala new file mode 100644 index 000000000000..d751f20ae3f6 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenFilteredScanSuite extends FilteredScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala new file mode 100644 index 000000000000..ad91b92aae20 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** Unit test suites for data source filters. */ +class GlutenFiltersSuite extends FiltersSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala new file mode 100644 index 000000000000..0437e29a7d35 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -0,0 +1,610 @@ +/* + * 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.spark.sql.sources + +import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.executor.OutputMetrics +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.{CommandResultExec, GlutenImplicits, QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.hadoop.fs.{Path, RawLocalFileSystem} + +import java.io.{File, IOException} + +class GlutenInsertSuite + extends InsertSuite + with GlutenSQLTestsBaseTrait + with AdaptiveSparkPlanHelper + with GlutenColumnarWriteTestSupport { + + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.leafNodeDefaultParallelism", "1") + } + + override def beforeAll(): Unit = { + super.beforeAll() + spark.sql(""" + |CREATE TABLE source USING PARQUET AS + |SELECT cast(id as int) as c1, cast(id % 5 as string) c2 FROM range(100) + |""".stripMargin) + + spark.sql("INSERT INTO TABLE source SELECT 0, null") + spark.sql("INSERT INTO TABLE source SELECT 0, ''") + } + + override def afterAll(): Unit = { + spark.sql("DROP TABLE source") + super.afterAll() + } + + private def checkWriteFilesAndGetChild(df: DataFrame): (SparkPlan, SparkPlan) = { + val writeFiles = stripAQEPlan( + df.queryExecution.executedPlan + .asInstanceOf[CommandResultExec] + .commandPhysicalPlan).children.head + val child = checkWriteFilesAndGetChild(writeFiles) + (writeFiles, child) + } + + testGluten("insert partition table") { + withTable("pt", "pt2") { + spark.sql("CREATE TABLE pt (c1 int, c2 string) USING PARQUET PARTITIONED BY (pt string)") + spark.sql("CREATE TABLE pt2 (c1 int, c2 string) USING PARQUET PARTITIONED BY (pt string)") + + var taskMetrics: OutputMetrics = null + val taskListener = new SparkListener { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + taskMetrics = taskEnd.taskMetrics.outputMetrics + } + } + + var sqlMetrics: Map[String, SQLMetric] = null + val queryListener = new QueryExecutionListener { + override def onFailure(f: String, qe: QueryExecution, e: Exception): Unit = {} + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + qe.executedPlan match { + case dataWritingCommandExec: DataWritingCommandExec => + sqlMetrics = dataWritingCommandExec.cmd.metrics + case _ => + } + } + } + spark.sparkContext.addSparkListener(taskListener) + spark.listenerManager.register(queryListener) + try { + val df = + spark.sql("INSERT INTO TABLE pt partition(pt='a') SELECT * FROM VALUES(1, 'a'),(2, 'b')") + spark.sparkContext.listenerBus.waitUntilEmpty() + checkWriteFilesAndGetChild(df) + + assert(taskMetrics.bytesWritten > 0) + assert(taskMetrics.recordsWritten == 2) + assert(sqlMetrics("numParts").value == 1) + assert(sqlMetrics("numOutputRows").value == 2) + assert(sqlMetrics("numOutputBytes").value > 0) + assert(sqlMetrics("numFiles").value == 1) + + checkAnswer(spark.sql("SELECT * FROM pt"), Row(1, "a", "a") :: Row(2, "b", "a") :: Nil) + } finally { + spark.sparkContext.removeSparkListener(taskListener) + spark.listenerManager.unregister(queryListener) + } + + // check no fallback nodes + val df2 = spark.sql("INSERT INTO TABLE pt2 SELECT * FROM pt") + checkWriteFilesAndGetChild(df2) + val fallbackSummary = GlutenImplicits + .collectQueryExecutionFallbackSummary(spark, df2.queryExecution) + assert(fallbackSummary.numFallbackNodes == 0) + } + } + + ignoreGluten("Cleanup staging files if job failed") { + // Using a unique table name in this test. Sometimes, the table is not removed for some unknown + // reason, which can cause test failure (location already exists) if other following tests have + // the same table name. + withTable("tbl") { + spark.sql("CREATE TABLE tbl (c1 int, c2 string) USING PARQUET") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("tbl")) + assert(new File(table.location).list().length == 0) + + intercept[Exception] { + spark.sql( + """ + |INSERT INTO TABLE tbl + |SELECT id, assert_true(SPARK_PARTITION_ID() = 1) FROM range(1, 3, 1, 2) + |""".stripMargin + ) + } + assert(new File(table.location).list().length == 0) + } + } + + private def validateDynamicPartitionWrite( + df: DataFrame, + expectedPartitionNames: Set[String]): Unit = { + val (writeFiles, writeChild) = checkWriteFilesAndGetChild(df) + assert( + writeFiles + .find(_.isInstanceOf[SortExecTransformer]) + .isEmpty) + // all operators should be transformed + assert(writeChild.find(!_.isInstanceOf[GlutenPlan]).isEmpty) + + val parts = spark.sessionState.catalog.listPartitionNames(TableIdentifier("pt")).toSet + assert(parts == expectedPartitionNames) + } + + testGluten("remove v1writes sort and project") { + // Only string type has empty2null expression + withTable("pt") { + spark.sql("CREATE TABLE pt (c1 int) USING PARQUET PARTITIONED BY(p string)") + + val df = spark.sql(s""" + |INSERT OVERWRITE TABLE pt PARTITION(p) + |SELECT c1, c2 as p FROM source + |""".stripMargin) + validateDynamicPartitionWrite( + df, + Set("p=0", "p=1", "p=2", "p=3", "p=4", "p=__HIVE_DEFAULT_PARTITION__")) + + // The partition column should never be empty + checkAnswer( + spark.sql("SELECT * FROM pt"), + spark.sql("SELECT c1, if(c2 = '', null, c2) FROM source")) + } + } + + testGluten("remove v1writes sort") { + // __HIVE_DEFAULT_PARTITION__ for other types are covered by other tests. + Seq( + ("p boolean", "coalesce(cast(c2 as boolean), false)", Set("p=false", "p=true")), + ("p short", "coalesce(cast(c2 as short), 0s)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ("p int", "coalesce(cast(c2 as int), 0)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ("p long", "coalesce(cast(c2 as long), 0l)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ( + "p date", + "if(c2 < 3, date '2023-01-01', date '2024-01-01')", + Set("p=2023-01-01", "p=2024-01-01")), + ( + "p int, p2 string", + "if(cast(c2 as int) < 2, 0, 1), c2", + Set( + "p=0/p2=1", + "p=0/p2=0", + "p=1/p2=__HIVE_DEFAULT_PARTITION__", + "p=1/p2=2", + "p=1/p2=3", + "p=1/p2=4" + )) + ) + .foreach { + case (partitionType, partitionExpr, expectedPartitionNames) => + withTable("pt") { + spark.sql(s"CREATE TABLE pt (c1 int) USING PARQUET PARTITIONED BY($partitionType)") + + val df = spark.sql(s""" + |INSERT OVERWRITE TABLE pt + |SELECT c1, $partitionExpr FROM source + |""".stripMargin) + validateDynamicPartitionWrite(df, expectedPartitionNames) + } + } + } + + testGluten("do not remove non-v1writes sort and project") { + withTable("t") { + spark.sql("CREATE TABLE t (c1 int, c2 string) USING PARQUET") + + val df = spark.sql("INSERT OVERWRITE TABLE t SELECT c1, c2 FROM source SORT BY c1") + val (writeFiles, _) = checkWriteFilesAndGetChild(df) + assert(writeFiles.find(x => x.isInstanceOf[SortExecTransformer]).isDefined) + checkAnswer(spark.sql("SELECT * FROM t"), spark.sql("SELECT * FROM source SORT BY c1")) + } + } + + testGluten("SPARK-35106: Throw exception when rename custom partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[ + GlutenRenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true") { + withTempPath { + path => + withTable("t") { + sql(""" + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + + val e = intercept[IOException] { + sql(s"insert into t partition(part1=1, part2=1) select 1") + } + assert(e.getMessage.contains("Failed to rename")) + } + } + } + } + + testGluten("Do not fallback write files if output columns contain Spark internal metadata") { + withTable("t1", "t2") { + spark.sql("CREATE TABLE t1 USING PARQUET AS SELECT id as c1, id % 3 as c2 FROM range(10)") + spark.sql("CREATE TABLE t2 (c1 long, c2 long) USING PARQUET") + val df = spark.sql("INSERT INTO TABLE t2 SELECT c2, count(*) FROM t1 GROUP BY c2") + checkWriteFilesAndGetChild(df) + } + } + + testGluten("Add metadata white list to allow native write files") { + withTable("t1", "t2") { + spark.sql(""" + |CREATE TABLE t1 (c1 long comment 'data column1', c2 long comment 'data column2') + |USING PARQUET + |""".stripMargin) + spark.sql("INSERT INTO TABLE t1 VALUES(1, 1),(2, 2)") + spark.sql("CREATE TABLE t2 (c1 long, c2 long) USING PARQUET") + val df = spark.sql("INSERT INTO TABLE t2 SELECT * FROM t1") + checkWriteFilesAndGetChild(df) + } + } + + testGluten("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") { + import testImplicits._ + case class Config(sqlConf: Option[(String, String)], useDataFrames: Boolean = false) + def runTest(dataSource: String, config: Config): Unit = { + def insertIntoT(): Unit = { + sql("insert into t(a, i) values('xyz', 42)") + } + def withTableT(f: => Unit): Unit = { + sql(s"create table t(a string, i int) using $dataSource") + withTable("t")(f) + } + // Positive tests: + // Adding a column with a valid default value into a table containing existing data + // returns null while it works successfully for newly added rows in Velox. + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + checkAnswer(spark.table("t"), Row("xyz", 42, "abcdef")) + checkAnswer(sql("select i, s from t"), Row(42, "abcdef")) + // Now alter the column to change the default value. + // This still returns the previous value, not the new value. + sql("alter table t alter column s set default concat('ghi', 'jkl')") + checkAnswer(sql("select i, s from t"), Row(42, "abcdef")) + } + // Adding a column with a default value and then inserting explicit NULL values works. + // Querying data back from the table differentiates between the explicit NULL values and + // default values. + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + if (config.useDataFrames) { + Seq((null, null, null)).toDF.write.insertInto("t") + } else { + sql("insert into t values(null, null, null)") + } + + checkAnswer(spark.table("t"), Seq(Row("xyz", 42, "abcdef"), Row(null, null, null))) + checkAnswer(sql("select i, s from t"), Seq(Row(42, "abcdef"), Row(null, null))) + } + // Adding two columns where only the first has a valid default value works successfully. + // Querying data from the altered table returns the default value as well as NULL for the + // second column.+ + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + sql("alter table t add column (x string)") + checkAnswer(spark.table("t"), Row("xyz", 42, "abcdef", null)) + checkAnswer(sql("select i, s, x from t"), Row(42, "abcdef", null)) + } + // Test other supported data types. + withTableT { + sql( + "alter table t add columns (" + + "s boolean default true, " + + "t byte default cast(null as byte), " + + "u short default cast(42 as short), " + + "v float default 0, " + + "w double default 0, " + + "x date default cast('2021-01-02' as date), " + + "y timestamp default cast('2021-01-02 01:01:01' as timestamp), " + + "z timestamp_ntz default cast('2021-01-02 01:01:01' as timestamp_ntz), " + + "a1 timestamp_ltz default cast('2021-01-02 01:01:01' as timestamp_ltz), " + + "a2 decimal(5, 2) default 123.45," + + "a3 bigint default 43," + + "a4 smallint default cast(5 as smallint)," + + "a5 tinyint default cast(6 as tinyint))") + insertIntoT() + // Manually inspect the result row values rather than using the 'checkAnswer' helper method + // in order to ensure the values' correctness while avoiding minor type incompatibilities. + val result: Array[Row] = + sql("select s, t, u, v, w, x, y, z, a1, a2, a3, a4, a5 from t").collect() + for (row <- result) { + assert(row.length == 13) + assert(row(0) == true) + assert(row(1) == null) + assert(row(2) == 42) + assert(row(3) == 0.0f) + assert(row(4) == 0.0d) + assert(row(5).toString == "2021-01-02") + assert(row(6).toString == "2021-01-02 01:01:01.0") + assert(row(7).toString.startsWith("2021-01-02")) + assert(row(8).toString == "2021-01-02 01:01:01.0") + assert(row(9).toString == "123.45") + assert(row(10) == 43L) + assert(row(11) == 5) + assert(row(12) == 6) + } + } + } + + // This represents one test configuration over a data source. + case class TestCase(dataSource: String, configs: Seq[Config]) + // Run the test several times using each configuration. + Seq( + TestCase( + dataSource = "csv", + Seq(Config(None), Config(Some(SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> "false")))), + TestCase( + dataSource = "json", + Seq(Config(None), Config(Some(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS.key -> "false")))), + TestCase( + dataSource = "orc", + Seq(Config(None), Config(Some(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")))), + TestCase( + dataSource = "parquet", + Seq(Config(None), Config(Some(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")))) + ).foreach { + testCase: TestCase => + testCase.configs.foreach { + config: Config => + // Run the test twice, once using SQL for the INSERT operations + // and again using DataFrames. + for (useDataFrames <- Seq(false, true)) { + config.sqlConf + .map { + kv: (String, String) => + withSQLConf(kv) { + // Run the test with the pair of custom SQLConf values. + runTest(testCase.dataSource, config.copy(useDataFrames = useDataFrames)) + } + } + .getOrElse { + // Run the test with default settings. + runTest(testCase.dataSource, config.copy(useDataFrames = useDataFrames)) + } + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with array defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + import testImplicits._ + // Positive tests: array types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql("alter table t add column s array default array(1, 2)") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) values (true)") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Seq(1, 2)))) + } + } + // Negative tests: provided array element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s array default array('abc', 'def')") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with struct defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + + import testImplicits._ + // Positive tests: struct types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql( + "alter table t add column s struct default struct(true, 'abc')") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) values (true)") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Row(true, "abc")))) + } + } + + // Negative tests: provided map element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s struct default struct(42, 56)") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } + + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with map defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + + import testImplicits._ + // Positive tests: map types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql("alter table t add column s map default map(true, 'abc')") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) select true") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Map(true -> "abc")))) + } + withTable("t") { + sql(s""" + create table t( + i int, + s struct< + x array< + struct>, + y array< + map>> + default struct( + array( + struct(1, 2)), + array( + map(false, 'def', true, 'jkl')))) + using ${config.dataSource}""") + sql("insert into t select 1, default") + sql("alter table t alter column s drop default") + if (config.useDataFrames) { + Seq((2, null)).toDF.write.insertInto("t") + } else { + sql("insert into t select 2, default") + } + sql(""" + alter table t alter column s + set default struct( + array( + struct(3, 4)), + array( + map(false, 'mno', true, 'pqr')))""") + sql("insert into t select 3, default") + sql(""" + alter table t + add column t array< + map> + default array( + map(true, 'xyz'))""") + sql("insert into t(i, s) select 4, default") + checkAnswer( + spark.table("t"), + Seq( + Row(1, Row(Seq(Row(1, 2)), Seq(Map(false -> "def", true -> "jkl"))), null), + Row(2, null, null), + Row(3, Row(Seq(Row(3, 4)), Seq(Map(false -> "mno", true -> "pqr"))), null), + Row( + 4, + Row(Seq(Row(3, 4)), Seq(Map(false -> "mno", true -> "pqr"))), + Seq(Map(true -> "xyz"))) + ) + ) + } + } + // Negative tests: provided map element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s map default map(42, 56)") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } +} + +class GlutenRenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + (!isSparkStagingDir(src) || isSparkStagingDir(dst)) && super.rename(src, dst) + } + + private def isSparkStagingDir(path: Path): Boolean = { + path.toString.contains("_temporary") + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala new file mode 100644 index 000000000000..26c847ff2323 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPartitionedWriteSuite extends PartitionedWriteSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala new file mode 100644 index 000000000000..94171f44cecc --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathOptionSuite extends PathOptionSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala new file mode 100644 index 000000000000..920d4f3af647 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenPrunedScanSuite extends PrunedScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala new file mode 100644 index 000000000000..ddd06bb3fd89 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenResolvedDataSourceSuite extends ResolvedDataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala new file mode 100644 index 000000000000..5ae0204b835d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSaveLoadSuite extends SaveLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala new file mode 100644 index 000000000000..ebd17781ff2b --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenTableScanSuite extends TableScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala new file mode 100644 index 000000000000..5bf53e66e642 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -0,0 +1,218 @@ +/* + * 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.spark.sql.statistics + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.internal.SQLConf + +import scala.util.control.Breaks.{break, breakable} + +/** + * TODO: There are some false positive & false negative cases for some functions. For such + * situation, we need to use a suitable test sql to do the check. + */ +class SparkFunctionStatistics extends QueryTest { + + var spark: SparkSession = null + + protected def initializeSession(): Unit = { + if (spark == null) { + val sparkBuilder = SparkSession + .builder() + .appName("Gluten-UT") + .master(s"local[2]") + // Avoid static evaluation for literal input by spark catalyst. + .config( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key, + ConvertToLocalRelation.ruleName + + "," + ConstantFolding.ruleName + "," + NullPropagation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "org.apache.gluten.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } + } + + def extractQuery(examples: String): Seq[String] = { + examples + .split("\n") + .map(_.trim) + .filter(!_.isEmpty) + .filter(_.startsWith("> SELECT")) + .map(_.replace("> SELECT", "SELECT")) + } + + test(GlutenTestConstants.GLUTEN_TEST + "Run spark function statistics: ") { + initializeSession + val functionRegistry = spark.sessionState.functionRegistry + val sparkBuiltInFunctions = functionRegistry.listFunction() + // According to expressionsForTimestampNTZSupport in FunctionRegistry.scala, + // these functions are registered only for testing, not available for end users. + // Other functions like current_database is NOT necessarily offloaded to native. + val ignoreFunctions = Seq( + "get_fake_app_name", + "current_catalog", + "current_database", + "spark_partition_id", + "current_user", + "current_timezone") + val supportedFunctions = new java.util.ArrayList[String]() + val unsupportedFunctions = new java.util.ArrayList[String]() + val needInspectFunctions = new java.util.ArrayList[String]() + + for (func <- sparkBuiltInFunctions) { + val exprInfo = functionRegistry.lookupFunction(func).get + if (!ignoreFunctions.contains(exprInfo.getName)) { + val examples = extractQuery(exprInfo.getExamples) + if (examples.isEmpty) { + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("## Not found examples for " + exprInfo.getName) + // scalastyle:on println + } + var isSupported: Boolean = true + breakable { + for (example <- examples) { + var executedPlan: SparkPlan = null + try { + executedPlan = spark.sql(example).queryExecution.executedPlan + } catch { + case t: Throwable => + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("-- Need inspect " + exprInfo.getName) + println(exprInfo.getExamples) + // scalastyle:on println + break + } + val hasFallbackProject = executedPlan.find(_.isInstanceOf[ProjectExec]).isDefined + if (hasFallbackProject) { + isSupported = false + break + } + val hasGlutenPlan = executedPlan.find(_.isInstanceOf[GlutenPlan]).isDefined + if (!hasGlutenPlan) { + isSupported = false + break + } + break + } + } + if (isSupported && !needInspectFunctions.contains(exprInfo.getName)) { + supportedFunctions.add(exprInfo.getName) + } else if (!isSupported) { + unsupportedFunctions.add(exprInfo.getName) + } + } + } + // scalastyle:off println + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions: " + supportedFunctions.size()) + println("Unsupported functions: " + unsupportedFunctions.size()) + println("Need inspect functions: " + needInspectFunctions.size()) + // scalastyle:on println + // For correction. + val supportedCastAliasFunctions = Seq( + "boolean", + "tinyint", + "smallint", + "int", + "bigint", + "float", + "double", + "decimal", + "date", + "binary", + "string") + for (func <- supportedCastAliasFunctions) { + if (needInspectFunctions.contains(func)) { + needInspectFunctions.remove(func) + supportedFunctions.add(func) + } + } + + // For wrongly recognized unsupported case. + Seq("%", "ceil", "floor", "first", "first_value", "last", "last_value", "hash", "mod").foreach( + name => { + if (unsupportedFunctions.remove(name)) { + supportedFunctions.add(name) + } + }) + // For wrongly recognized supported case. + Seq( + "array_contains", + "map_keys", + "get_json_object", + "element_at", + "map_from_arrays", + "contains", + "startswith", + "endswith", + "map_contains_key", + "map_values", + "try_element_at", + "struct", + "array", + "ilike", + "sec", + "csc" + ).foreach( + name => { + if (supportedFunctions.remove(name)) { + unsupportedFunctions.add(name) + } + }) + // Functions in needInspectFunctions were checked. + unsupportedFunctions.addAll(needInspectFunctions) + // scalastyle:off println + println("---------------") + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions corrected: " + supportedFunctions.size()) + println("Unsupported functions corrected: " + unsupportedFunctions.size()) + println("Support list:") + println(supportedFunctions) + println("Not support list:") + println(unsupportedFunctions) + // scalastyle:on println + } +} diff --git a/gluten-ut/test/pom.xml b/gluten-ut/test/pom.xml index fd525abed65e..5867434c7483 100644 --- a/gluten-ut/test/pom.xml +++ b/gluten-ut/test/pom.xml @@ -36,13 +36,6 @@ test-jar compile - - org.apache.parquet - parquet-column - 1.13.1 - tests - test - diff --git a/pom.xml b/pom.xml index 6e2c68276555..604c424a1512 100644 --- a/pom.xml +++ b/pom.xml @@ -1167,9 +1167,11 @@ 40 0.15.0 - 2.15.1 - 3.3.4 - 4.9.3 + 2.18.2 + 2.18.2 + 2.18.2 + 3.4.1 + 4.13.1 33.4.0-jre 2.0.16 2.24.3 diff --git a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala index 329ac65e76f9..ef2dee605f3d 100644 --- a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala +++ b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala @@ -352,6 +352,7 @@ object ExpressionNames { final val KNOWN_NULLABLE = "known_nullable" final val KNOWN_NOT_NULL = "known_not_null" final val KNOWN_FLOATING_POINT_NORMALIZED = "known_floating_point_normalized" + final val KNOWN_NOT_CONTAINS_NULL = "known_not_contains_null" final val NORMALIZE_NANAND_ZERO = "normalize_nanand_zero" // Window functions used by Substrait plan. diff --git a/shims/spark40/src/main/scala/org/apache/gluten/execution/PartitionedFileUtilShim.scala b/shims/spark40/src/main/scala/org/apache/gluten/execution/PartitionedFileUtilShim.scala index 838932add717..250ad6cb88a8 100644 --- a/shims/spark40/src/main/scala/org/apache/gluten/execution/PartitionedFileUtilShim.scala +++ b/shims/spark40/src/main/scala/org/apache/gluten/execution/PartitionedFileUtilShim.scala @@ -57,6 +57,22 @@ object PartitionedFileUtilShim { } } + private lazy val getPartitionedFileByPathSizeMethod: Method = { + try { + val m = clz.getDeclaredMethod( + "getPartitionedFile", + classOf[FileStatusWithMetadata], + classOf[Path], + classOf[InternalRow], + classOf[Long], + classOf[Long]) + m.setAccessible(true) + m + } catch { + case _: NoSuchMethodException => null + } + } + def getPartitionedFile( file: FileStatusWithMetadata, partitionValues: InternalRow): PartitionedFile = { @@ -68,6 +84,10 @@ object PartitionedFileUtilShim { getPartitionedFileByPathMethod .invoke(module, file, file.getPath, partitionValues) .asInstanceOf[PartitionedFile] + } else if (getPartitionedFileByPathSizeMethod != null) { + getPartitionedFileByPathSizeMethod + .invoke(module, file, file.getPath, partitionValues, 0, file.getLen) + .asInstanceOf[PartitionedFile] } else { val params = clz.getDeclaredMethods .find(_.getName == "getPartitionedFile") diff --git a/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala b/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala index 11d04198ab77..21bc6c05710b 100644 --- a/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala +++ b/shims/spark40/src/main/scala/org/apache/gluten/sql/shims/spark40/Spark40Shims.scala @@ -88,12 +88,14 @@ class Spark40Shims extends SparkShims { Sig[Empty2Null](ExpressionNames.EMPTY2NULL), Sig[Mask](ExpressionNames.MASK), Sig[TimestampAdd](ExpressionNames.TIMESTAMP_ADD), + Sig[TimestampDiff](ExpressionNames.TIMESTAMP_DIFF), Sig[RoundFloor](ExpressionNames.FLOOR), Sig[RoundCeil](ExpressionNames.CEIL), Sig[ArrayInsert](ExpressionNames.ARRAY_INSERT), Sig[CheckOverflowInTableInsert](ExpressionNames.CHECK_OVERFLOW_IN_TABLE_INSERT), Sig[ArrayAppend](ExpressionNames.ARRAY_APPEND), Sig[UrlEncode](ExpressionNames.URL_ENCODE), + Sig[KnownNotContainsNull](ExpressionNames.KNOWN_NOT_CONTAINS_NULL), Sig[UrlDecode](ExpressionNames.URL_DECODE) ) } @@ -700,6 +702,22 @@ class Spark40Shims extends SparkShims { plan.offset } + override def extractExpressionTimestampAddUnit(exp: Expression): Option[Seq[String]] = { + exp match { + case timestampAdd: TimestampAdd => + Option.apply(Seq(timestampAdd.unit, timestampAdd.timeZoneId.getOrElse(""))) + case _ => Option.empty + } + } + + override def extractExpressionTimestampDiffUnit(exp: Expression): Option[String] = { + exp match { + case timestampDiff: TimestampDiff => + Some(timestampDiff.unit) + case _ => Option.empty + } + } + override def widerDecimalType(d1: DecimalType, d2: DecimalType): DecimalType = { DecimalPrecisionTypeCoercion.widerDecimalType(d1, d2) }