diff --git a/.github/workflows/util/install-spark-resources.sh b/.github/workflows/util/install-spark-resources.sh index a00c684a03a8..5988f3836287 100755 --- a/.github/workflows/util/install-spark-resources.sh +++ b/.github/workflows/util/install-spark-resources.sh @@ -80,7 +80,8 @@ function install_spark() { mv python ${INSTALL_DIR}/shims/spark"${spark_version_short}""${scala_suffix}"/spark_home mv bin ${INSTALL_DIR}/shims/spark"${spark_version_short}""${scala_suffix}"/spark_home - tar --strip-components=1 -xf "${local_source}" spark-"${spark_version}"/sql/core/src/test/resources/ + tar --strip-components=1 -xf "${local_source}" spark-"${spark_version}"/sql/core/src/test/resources/ \ + spark-"${spark_version}"/sql/hive/src/test/resources/ mkdir -p shims/spark"${spark_version_short}${scala_suffix}"/spark_home/ mv sql shims/spark"${spark_version_short}${scala_suffix}"/spark_home/ diff --git a/.github/workflows/velox_backend_enhanced.yml b/.github/workflows/velox_backend_enhanced.yml index 7cf24f0879a4..f234ef59ab2d 100644 --- a/.github/workflows/velox_backend_enhanced.yml +++ b/.github/workflows/velox_backend_enhanced.yml @@ -130,7 +130,7 @@ jobs: export SPARK_HOME=/opt/shims/spark34/spark_home/ ls -l $SPARK_HOME $MVN_CMD clean test -Pspark-3.4 -Pjava-17 -Pbackends-velox -Piceberg -Piceberg-test -Pdelta -Phudi \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTest \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTest \ -DargLine="-Dspark.test.home=$SPARK_HOME" - name: Upload test report if: always() @@ -188,7 +188,7 @@ jobs: export SPARK_HOME=/opt/shims/spark35/spark_home/ ls -l $SPARK_HOME $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTest \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTest \ -DargLine="-Dspark.test.home=$SPARK_HOME" - name: Upload test report if: always() @@ -215,6 +215,9 @@ jobs: spark-test-spark35-slow: needs: build-native-lib-centos-7 runs-on: ubuntu-22.04 + env: #TODO remove after image update + LANG: C.UTF-8 + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -228,6 +231,10 @@ jobs: with: name: arrow-jars-enhanced-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 3.5.5 #TODO remove after image update + run: | + rm -rf /opt/shims/spark35 + bash .github/workflows/util/install-spark-resources.sh 3.5 - name: Build and Run unit test for Spark 3.5.5 (slow tests) run: | cd $GITHUB_WORKSPACE/ @@ -238,6 +245,9 @@ jobs: $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 @@ -293,7 +303,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta \ -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.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 diff --git a/.github/workflows/velox_backend_x86.yml b/.github/workflows/velox_backend_x86.yml index f130a7f62e6c..1e4b6fb24afc 100644 --- a/.github/workflows/velox_backend_x86.yml +++ b/.github/workflows/velox_backend_x86.yml @@ -680,6 +680,8 @@ jobs: spark-test-spark33: needs: build-native-lib-centos-7 runs-on: ubuntu-22.04 + env: + SPARK_TESTING: true container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -710,7 +712,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-3.3 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark33/spark_home/" \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,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 @@ -736,6 +738,10 @@ jobs: spark-test-spark33-slow: needs: build-native-lib-centos-7 runs-on: ubuntu-22.04 + env: + SPARK_TESTING: true + LANG: C.UTF-8 #TODO remove after image update + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -749,6 +755,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 3.3.1 #TODO remove after image update + run: | + rm -rf /opt/shims/spark33 + bash .github/workflows/util/install-spark-resources.sh 3.3 - name: Build and Run unit test for Spark 3.3.1 (slow tests) run: | @@ -760,6 +770,9 @@ jobs: $MVN_CMD clean test -Pspark-3.3 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark33/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-3.3 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark33/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 @@ -810,7 +823,7 @@ jobs: export SPARK_HOME=/opt/shims/spark34/spark_home/ ls -l $SPARK_HOME $MVN_CMD clean test -Pspark-3.4 -Pjava-17 -Pbackends-velox -Piceberg -Piceberg-test -Pdelta -Phudi -Ppaimon -Pspark-ut \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest \ + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest \ -DargLine="-Dspark.test.home=$SPARK_HOME" - name: Upload test report if: always() @@ -837,6 +850,9 @@ jobs: spark-test-spark34-slow: needs: build-native-lib-centos-7 runs-on: ubuntu-22.04 + env: #TODO remove after image update + LANG: C.UTF-8 + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -850,6 +866,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 3.4.4 #TODO remove after image update + run: | + rm -rf /opt/shims/spark34 + bash .github/workflows/util/install-spark-resources.sh 3.4 - name: Build and Run unit test for Spark 3.4.4 (slow tests) run: | cd $GITHUB_WORKSPACE/ @@ -862,6 +882,9 @@ jobs: $MVN_CMD clean test -Pspark-3.4 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest \ -DargLine="-Dspark.test.home=$SPARK_HOME" + $MVN_CMD clean test -Pspark-3.4 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest \ + -DargLine="-Dspark.test.home=$SPARK_HOME" - name: Upload test report if: always() uses: actions/upload-artifact@v4 @@ -913,7 +936,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/" \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,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 @@ -971,7 +994,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-3.5 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Piceberg \ -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark35-scala-2.13/spark_home/" \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,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 @@ -993,6 +1016,8 @@ jobs: runs-on: ubuntu-22.04 env: SPARK_TESTING: true + LANG: C.UTF-8 #TODO remove after image update + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -1006,6 +1031,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 3.5.5 #TODO remove after image update + run: | + rm -rf /opt/shims/spark35 + bash .github/workflows/util/install-spark-resources.sh 3.5 - name: Build and Run unit test for Spark 3.5.5 (slow tests) run: | cd $GITHUB_WORKSPACE/ @@ -1016,6 +1045,9 @@ jobs: $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Phudi -Ppaimon -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 @@ -1067,7 +1099,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/ -Dspark.gluten.sql.columnar.forceShuffledHashJoin=false" \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest - name: Upload test report uses: actions/upload-artifact@v4 with: @@ -1088,6 +1120,8 @@ jobs: runs-on: ubuntu-22.04 env: SPARK_TESTING: true + LANG: C.UTF-8 #TODO remove after image update + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk8 steps: - uses: actions/checkout@v4 @@ -1101,6 +1135,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ + - name: Prepare Spark Resources for Spark 3.5.5 #TODO remove after image update + run: | + rm -rf /opt/shims/spark35 + bash .github/workflows/util/install-spark-resources.sh 3.5 - name: Build and Run unit test for Spark 3.5.5 (slow tests) run: | cd $GITHUB_WORKSPACE/ @@ -1111,6 +1149,9 @@ jobs: $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/ -Dspark.gluten.sql.columnar.forceShuffledHashJoin=false" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-3.5 -Pjava-17 -Pbackends-velox -Piceberg -Pdelta -Ppaimon -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark35/spark_home/ -Dspark.gluten.sql.columnar.forceShuffledHashJoin=false" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report uses: actions/upload-artifact@v4 with: @@ -1281,7 +1322,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pdelta -Ppaimon \ -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 + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,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 @@ -1303,6 +1344,8 @@ jobs: runs-on: ubuntu-22.04 env: SPARK_TESTING: true + LANG: C.UTF-8 #TODO remove after image update + LC_ALL: C.UTF-8 container: apache/gluten:centos-8-jdk17 steps: - uses: actions/checkout@v4 @@ -1330,6 +1373,9 @@ jobs: $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pdelta -Ppaimon -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark40/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-4.0 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pdelta -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark40/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 @@ -1387,7 +1433,7 @@ jobs: java -version $MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox \ -Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \ - -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest + -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,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 @@ -1409,6 +1455,8 @@ jobs: runs-on: ubuntu-22.04 env: SPARK_TESTING: true + LANG: C.UTF-8 #TODO remove after image update + LC_ALL: C.UTF-8 container: apache/gluten:centos-9-jdk17 steps: - uses: actions/checkout@v4 @@ -1436,6 +1484,9 @@ jobs: $MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pspark-ut \ -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest + $MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pspark-ut \ + -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \ + -DtagsToInclude=org.apache.spark.tags.SlowHiveTest - name: Upload test report if: always() uses: actions/upload-artifact@v4 diff --git a/dev/docker/Dockerfile.centos8-dynamic-build b/dev/docker/Dockerfile.centos8-dynamic-build index 48f45794a49f..90435d04125f 100644 --- a/dev/docker/Dockerfile.centos8-dynamic-build +++ b/dev/docker/Dockerfile.centos8-dynamic-build @@ -23,6 +23,8 @@ ARG JAVA_VERSION=1.8.0 ENV JAVA_HOME=/usr/lib/jvm/java-${JAVA_VERSION}-openjdk ENV PATH=$JAVA_HOME/bin:$PATH +ENV LANG=C.UTF-8 +ENV LC_ALL=C.UTF-8 RUN set -ex; \ sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true; \ diff --git a/dev/docker/Dockerfile.centos9-dynamic-build b/dev/docker/Dockerfile.centos9-dynamic-build index 7b3387ce8c83..980737df7b18 100644 --- a/dev/docker/Dockerfile.centos9-dynamic-build +++ b/dev/docker/Dockerfile.centos9-dynamic-build @@ -23,6 +23,8 @@ ARG JAVA_VERSION=1.8.0 ENV JAVA_HOME=/usr/lib/jvm/java-${JAVA_VERSION}-openjdk ENV PATH=$JAVA_HOME/bin:$PATH +ENV LANG=C.UTF-8 +ENV LC_ALL=C.UTF-8 RUN set -ex; \ yum update -y && yum install -y epel-release sudo dnf && yum install -y ccache; \ diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 62634fc7dab0..cb795fd7c7cd 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.joins.{GlutenBroadcastJoinSuite, GlutenExi import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.GlutenFallbackSuite -import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.sources._ // Some settings' line length exceeds 100 @@ -829,7 +829,7 @@ class VeloxTestSettings extends BackendTestSettings { // enableSuite[GlutenSimpleShowCreateTableSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] - enableSuite[GlutenSQLQuerySuite] + enableSuite[org.apache.spark.sql.GlutenSQLQuerySuite] // Decimal precision exceeds. .exclude("should be able to resolve a persistent view") // Unstable. Needs to be fixed. @@ -867,7 +867,38 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("cases when literal is max") enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHashAggregationQuerySuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHashAggregationQueryWithControlledFallbackSuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHiveCommandSuite] + enableSuite[GlutenHiveDDLSuite] + enableSuite[GlutenHiveExplainSuite] + .exclude("explain output of physical plan should contain proper codegen stage ID") + .exclude("EXPLAIN CODEGEN command") + enableSuite[GlutenHivePlanTest] + enableSuite[GlutenHiveQuerySuite] + enableSuite[GlutenHiveResolutionSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenHiveSQLViewSuite] + enableSuite[GlutenHiveScriptTransformationSuite] + enableSuite[GlutenHiveSerDeReadWriteSuite] + enableSuite[GlutenHiveSerDeSuite] + enableSuite[GlutenHiveTableScanSuite] + enableSuite[GlutenHiveTypeCoercionSuite] + enableSuite[GlutenHiveUDAFSuite] + enableSuite[GlutenHiveUDFSuite] + enableSuite[GlutenObjectHashAggregateSuite] + enableSuite[GlutenPruneHiveTablePartitionsSuite] + enableSuite[GlutenPruningSuite] + enableSuite[GlutenSQLMetricsSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLQuerySuite] + enableSuite[GlutenHashUDAQuerySuite] + enableSuite[GlutenHashUDAQueryWithControlledFallbackSuite] + enableSuite[GlutenSQLQuerySuiteAE] + enableSuite[GlutenWindowQuerySuite] enableSuite[GlutenImplicitsTest] enableSuite[GlutenCollapseProjectExecTransformerSuite] enableSuite[GlutenSparkSessionExtensionSuite] diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala new file mode 100644 index 000000000000..e8494d2c075f --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashAggregationQuerySuite + extends HashAggregationQuerySuite + with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashAggregationQueryWithControlledFallbackSuite + extends HashAggregationQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala new file mode 100644 index 000000000000..9ce9db428741 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File +import java.nio.file.Files +import java.nio.file.StandardCopyOption + +@SlowHiveTest +class GlutenHiveCommandSuite extends HiveCommandSuite with GlutenTestSetWithSystemPropertyTrait { + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + // Rewritten with a workspace-backed file because TestHive.getHiveFile resolves this resource + // from the spark-hive tests jar in gluten-ut. + "LOAD DATA LOCAL", + "LOAD DATA" + ) + + Seq(true, false).foreach { + local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + testGluten(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = getWorkspaceFilePath( + "sql", + "hive", + "src", + "test", + "resources", + "data", + "files", + "employee.dat").toFile.getCanonicalFile + + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData.toPath, tmp.toPath, StandardCopyOption.REPLACE_EXISTING) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql(""" + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { + path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + sql(""" + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { + f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table")) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala new file mode 100644 index 000000000000..d96078f2d557 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala @@ -0,0 +1,324 @@ +/* + * 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.sql.Dataset +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} + +import java.io.{File, FileOutputStream} +import java.util +import java.util.Locale + +import scala.util.control.NonFatal + +/** + * Mostly copied from Spark's [[HiveComparisonTest]]. This support reuses [[GlutenTestHiveTables]] + * for TestHive table registrations and customizes [[createQueryTest]] so `../../data` resolves to + * workspace-backed Hive test resources. + */ +trait GlutenHiveComparisonTestSupport extends GlutenHiveResourcePathSupport { + this: HiveComparisonTest => + + private val testDataPath: String = { + hiveResourcePath("data").toAbsolutePath.normalize.toUri.getPath + .stripSuffix("/") + } + + private val glutenAnswerCache: File = hiveResourcePath("golden").toFile + + override def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false): Unit = { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logDebug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + + test(testCaseName) { + assume(!skip) + logDebug(s"=== HIVE TEST: $testCaseName ===") + + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // the system to return the wrong answer. Since we have no intention of mirroring their + // previously broken behavior we simply filter out changes to this setting. + .filterNot(_.contains("hive.outerjoin.supports.filters")) + .filterNot(_.contains("hive.exec.post.hooks")) + + if (allQueries != queryList) { + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex + .map { + case (query, i) => + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" + } + .mkString("\n== Console version of this test ==\n", "\n", "\n") + } + + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + if (reset) { + TestHive.reset() + } + + // Register workspace-backed table definitions before lazy TestHive auto-loading kicks in. + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { + query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(glutenAnswerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles + .flatMap { + cachedAnswerFile => + logDebug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(fileToString(cachedAnswerFile)) + } else { + logDebug(s"File $cachedAnswerFile not found") + None + } + } + .map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logInfo(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { + case (queryString, hive) => + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) + } + try { (query, prepareAnswer(query, getResult())) } + catch { + case e: Throwable => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + """.stripMargin + stringToFile( + new File(failedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + queryList.zip(hiveResults).zip(catalystResults).foreach { + case ((query, hive), (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery, hive) + + // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction + if ( + (!hiveQuery.logical.isInstanceOf[ExplainCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && + preparedHive != catalyst + ) { + + val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logWarning(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = + try { + val tablesRead = + new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHiveQueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => + e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead.contains(i.table.identifier) => + (q, e, i) + } + } + + tablesGenerated + .map { + case (hiveql, execution, insert) => + val rdd = + Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${rdd.collect().mkString("\n")} + """.stripMargin + } + .mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + |$computedTablesMessages + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq("into table", "create table", "drop index") + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { + query => excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) + } + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala new file mode 100644 index 000000000000..049c3b97f774 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala @@ -0,0 +1,121 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveDDLSuite + extends HiveDDLSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed schema URLs because TestHive.getHiveFile resolves + // these resources from the spark-hive tests jar in gluten-ut. + "SPARK-34370: support Avro schema evolution (add column with avro.schema.url)", + "SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)", + "SPARK-34261: Avoid side effect if create exists temporary function" + ) + + testGluten("SPARK-34370: support Avro schema evolution (add column with avro.schema.url)") { + checkAvroSchemaEvolutionAddColumn( + avroSchemaUrlProperty("schemaWithOneField.avsc"), + avroSchemaUrlProperty("schemaWithTwoFields.avsc")) + } + + testGluten("SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)") { + checkAvroSchemaEvolutionRemoveColumn( + avroSchemaUrlProperty("schemaWithTwoFields.avsc"), + avroSchemaUrlProperty("schemaWithOneField.avsc")) + } + + testGluten("SPARK-34261: Avoid side effect if create exists temporary function") { + withUserDefinedFunction("f1" -> true) { + sql("CREATE TEMPORARY FUNCTION f1 AS 'org.apache.hadoop.hive.ql.udf.UDFUUID'") + + val jarName = "TestUDTF.jar" + val jar = hiveResourcePath(jarName).toUri.toString + spark.sparkContext.addedJars.keys + .find(_.contains(jarName)) + .foreach(spark.sparkContext.addedJars.remove) + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + val msg = intercept[AnalysisException] { + sql( + "CREATE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + }.getMessage + assert(msg.contains("Function f1 already exists")) + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + + sql( + "CREATE OR REPLACE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + assert(spark.sparkContext.listJars().exists(_.contains(jarName))) + } + } + + private def avroSchemaUrlProperty(fileName: String): String = { + val schemaPath = hiveResourcePath(fileName) + s"'avro.schema.url'='${schemaPath.toUri.toString}'" + } + + private def checkAvroSchemaEvolutionAddColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + checkAnswer( + spark.table("t"), + Row("col1_default", "col2_value", "1981-01-07") :: + Row("col1_value", "col2_value", "1983-04-27") :: Nil) + } + } + + private def checkAvroSchemaEvolutionRemoveColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + checkAnswer( + spark.table("t"), + Row("col2_value", "1981-01-07") :: + Row("col2_value", "1983-04-27") :: Nil) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala new file mode 100644 index 000000000000..0f6b37016691 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveExplainSuite extends HiveExplainSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala new file mode 100644 index 000000000000..6cdf5f5845da --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHivePlanTest extends HivePlanTest with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala new file mode 100644 index 000000000000..f547953c58a3 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala @@ -0,0 +1,111 @@ +/* + * 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.SparkFiles +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.HiveUtils.{builtinHiveVersion => hiveVersion} +import org.apache.spark.sql.hive.test.HiveTestJars +import org.apache.spark.tags.SlowHiveTest + +import java.io.File + +@SlowHiveTest +class GlutenHiveQuerySuite + extends HiveQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + "ADD FILE command", + "ADD JAR command 2", + "CREATE TEMPORARY FUNCTION", + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + testGluten("ADD FILE command") { + val testFile = hiveResourcePath("data/files/v1.txt").toFile.toURI + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { + _ => Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + assert(sql("list files").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) + } + + testGluten("ADD JAR command 2") { + val testJar = HiveTestJars.getHiveHcatalogCoreJar().toURI + val testData = hiveResourcePath("data/files/sample.json").toUri + sql(s"ADD JAR $testJar") + withTable("t1") { + sql("""CREATE TABLE t1(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") + sql("select * from src join t1 on src.key = t1.a") + } + assert( + sql("list jars") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + assert( + sql("list jar") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + val testJar2 = hiveResourcePath("TestUDTF.jar").toFile.getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) + } + + testGluten("SPARK-33084: Add jar support Ivy URI in SQL") { + val testData = hiveResourcePath("data/files/sample.json").toUri + withTable("t") { + // Use transitive=false as it should be good enough to test the Ivy support in Hive ADD JAR. + sql( + s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion" + + "?transitive=false") + sql("""CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert( + sql("LIST JARS") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + assert( + sql("LIST JAR") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + } + } + + testGluten("CREATE TEMPORARY FUNCTION") { + val jarURL = hiveResourcePath("TestUDTF.jar").toUri.toURL + sql(s"ADD JAR $jarURL") + withUserDefinedFunction("udtf_count2" -> true) { + sql("""CREATE TEMPORARY FUNCTION udtf_count2 AS + |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |""".stripMargin) + assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala new file mode 100644 index 000000000000..d46c2eed7ad3 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveResolutionSuite + extends HiveResolutionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala new file mode 100644 index 000000000000..41a20e8ae6da --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.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.hive.execution + +import org.apache.spark.SparkFunSuite + +import java.nio.file.Path + +/** Shared helpers for resolving workspace-backed Hive test resources. */ +trait GlutenHiveResourcePathSupport { + this: SparkFunSuite => + + protected lazy val hiveTestResourceDir: Path = + getWorkspaceFilePath("sql", "hive", "src", "test", "resources") + + final protected def hiveResourcePath(relativePath: String): Path = { + GlutenTestHiveTables.hiveResourcePath(hiveTestResourceDir, relativePath) + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala new file mode 100644 index 000000000000..d537a9aae6f3 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSQLViewSuite extends HiveSQLViewSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala new file mode 100644 index 000000000000..158ad4f46e64 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveScriptTransformationSuite + extends HiveScriptTransformationSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala new file mode 100644 index 000000000000..92703dea3711 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeReadWriteSuite + extends HiveSerDeReadWriteSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala new file mode 100644 index 000000000000..3687714a592d --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeSuite + extends HiveSerDeSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + /** + * Mostly copied from Spark's [[HiveSerDeSuite]] and [[GlutenTestSetWithSystemPropertyTrait]], and + * customized so `sales.txt` is loaded from the workspace-backed Hive test resources instead of + * via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + + TestHive.setCacheTables(false) + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"""LOAD DATA LOCAL INPATH '${hiveResourcePath("data/files/sales.txt").toFile.toURI}' + |INTO TABLE sales""".stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala new file mode 100644 index 000000000000..ac4903218fcf --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.Row +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTableScanSuite + extends HiveTableScanSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed file because the upstream test resolves this resource + // from the spark-hive tests jar in gluten-ut. + "Spark-4077: timestamp query for null value" + ) + + testGluten("Spark-4077: timestamp query for null value") { + withTable("timestamp_query_null") { + sql(""" + |CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY ',' + |LINES TERMINATED BY '\n' + """.stripMargin) + val location = hiveResourcePath("data/files/issue-4077-data.txt").toFile.toURI + + sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert( + sql("SELECT time FROM timestamp_query_null LIMIT 2").collect() === + Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")), Row(null))) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala new file mode 100644 index 000000000000..d77f2c7d5780 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTypeCoercionSuite + extends HiveTypeCoercionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala new file mode 100644 index 000000000000..975fc9fb8997 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDAFSuite extends HiveUDAFSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala new file mode 100644 index 000000000000..a196da6ab4f4 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDFSuite + extends HiveUDFSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + "UDTF", + "permanent UDTF" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("UDTF") { + withUserDefinedFunction("udtf_count2" -> true) { + sql(s"ADD JAR ${hiveResourcePath("TestUDTF.jar").toFile.getCanonicalPath}") + sql(""" + |CREATE TEMPORARY FUNCTION udtf_count2 + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count2(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } + } + + testGluten("permanent UDTF") { + withUserDefinedFunction("udtf_count_temp" -> false) { + sql(s""" + |CREATE FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count_temp(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..327b633b1f0e --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenObjectHashAggregateSuite + extends ObjectHashAggregateSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala new file mode 100644 index 000000000000..14ffda794dcf --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruneHiveTablePartitionsSuite + extends PruneHiveTablePartitionsSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala new file mode 100644 index 000000000000..aeb6e85b6a0c --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruningSuite + extends PruningSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala new file mode 100644 index 000000000000..df750c09c49c --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenSQLMetricsSuite extends SQLMetricsSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..2b9272c7063d --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala @@ -0,0 +1,141 @@ +/* + * 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.TestUtils +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.sql.Date + +trait GlutenSQLQuerySuiteBase + extends SQLQuerySuiteBase + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed jar paths because TestHive.getHiveFile resolves + // these jars from the Spark test jars in gluten-ut. + "script", + "describe functions - user defined functions", + "describe functions - temporary user defined functions", + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("script") { + withTempView("script_table") { + import spark.implicits._ + + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) + val scriptFilePath = hiveResourcePath("test_script.sh").toFile.getCanonicalPath + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query = sql(s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + } + + testGluten("describe functions - user defined functions") { + withUserDefinedFunction("udtf_count" -> false) { + sql(s""" + |CREATE FUNCTION udtf_count + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + checkAnswer( + sql("SELECT udtf_count(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + } + } + + testGluten("describe functions - temporary user defined functions") { + withUserDefinedFunction("udtf_count_temp" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + } + } + + testGluten( + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") { + withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack1 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack2 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack2' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + + Seq("udtf_stack1", "udtf_stack2").foreach { + udf => + checkAnswer( + sql(s"SELECT $udf(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')"), + Seq(Row("A", 10, Date.valueOf("2015-01-01")), Row("B", 20, Date.valueOf("2016-01-01"))) + ) + } + } + } +} + +@SlowHiveTest +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLQuerySuiteBase {} + +@SlowHiveTest +class GlutenSQLQuerySuiteAE extends SQLQuerySuiteAE with GlutenSQLQuerySuiteBase {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala new file mode 100644 index 000000000000..0eaa597a8680 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive.sparkSession +import org.apache.spark.sql.hive.test.TestHive.sparkSession.TestTable +import org.apache.spark.sql.hive.test.TestHiveQueryExecution + +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + +import java.nio.file.Path + +/** Wrappers around [[TestHive]] lazy-loaded table registrations. */ +object GlutenTestHiveTables { + def hiveResourcePath(resourceDir: Path, relativePath: String): Path = { + relativePath.split('/').foldLeft(resourceDir) { case (path, child) => path.resolve(child) } + } + + private def hiveDataFile(resourceDir: Path, fileName: String): String = { + hiveResourcePath(resourceDir, fileName).toAbsolutePath.normalize.toString + } + + implicit private class SqlCmd(sql: String) { + def cmd: () => Unit = { + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit + } + } + + def registerTestTable(testTable: TestTable): Unit = { + sparkSession.registerTestTable(testTable) + } + + def registerHiveQTestUtilsTables(resourceDir: Path): Unit = { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable( + "src", + createTableSQL("src").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |INTO TABLE src""".stripMargin.cmd + ), + TestTable( + "src1", + createTableSQL("src1").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv3.txt")}' + |INTO TABLE src1""".stripMargin.cmd + ), + TestTable( + "srcpart", + () => { + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "srcpart1", + () => { + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "src_thrift", + () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() + + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + } + ), + TestTable( + "serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd + ), + TestTable( + "episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable( + "episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable( + "src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/json.txt")}' + |INTO TABLE src_json""".stripMargin.cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala new file mode 100644 index 000000000000..0d2798995580 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashUDAQuerySuite extends HashUDAQuerySuite with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashUDAQueryWithControlledFallbackSuite + extends HashUDAQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala new file mode 100644 index 000000000000..701fbb8467a9 --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala @@ -0,0 +1,70 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenWindowQuerySuite + extends WindowQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + /** + * Mostly copied from Spark's [[WindowQuerySuite]] and [[GlutenTestSetWithSystemPropertyTrait]], + * and customized so `part_tiny.txt` is loaded from the workspace-backed Hive test resources + * instead of via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + sql("DROP TABLE IF EXISTS part") + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) USING hive + """.stripMargin) + val testData1 = hiveResourcePath("data/files/part_tiny.txt").toFile.toURI + sql(s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index d457638e33b2..c4cf33738a5f 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.execution.joins.{GlutenBroadcastJoinSuite, GlutenExi import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.GlutenFallbackSuite -import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.sources.{GlutenBucketedReadWithoutHiveSupportSuite, GlutenBucketedWriteWithoutHiveSupportSuite, GlutenCreateTableAsSelectSuite, GlutenDDLSourceLoadSuite, GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite, GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE, GlutenExternalCommandRunnerSuite, GlutenFilteredScanSuite, GlutenFiltersSuite, GlutenInsertSuite, GlutenPartitionedWriteSuite, GlutenPathOptionSuite, GlutenPrunedScanSuite, GlutenResolvedDataSourceSuite, GlutenSaveLoadSuite, GlutenTableScanSuite} // Some settings' line length exceeds 100 @@ -871,7 +871,7 @@ class VeloxTestSettings extends BackendTestSettings { // enableSuite[GlutenSimpleShowCreateTableSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] - enableSuite[GlutenSQLQuerySuite] + enableSuite[org.apache.spark.sql.GlutenSQLQuerySuite] // Decimal precision exceeds. .exclude("should be able to resolve a persistent view") // Unstable. Needs to be fixed. @@ -911,7 +911,38 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("cases when literal is max") enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHashAggregationQuerySuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHashAggregationQueryWithControlledFallbackSuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHiveCommandSuite] + enableSuite[GlutenHiveDDLSuite] + enableSuite[GlutenHiveExplainSuite] + .exclude("explain output of physical plan should contain proper codegen stage ID") + .exclude("EXPLAIN CODEGEN command") + enableSuite[GlutenHivePlanTest] + enableSuite[GlutenHiveQuerySuite] + enableSuite[GlutenHiveResolutionSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenHiveSQLViewSuite] + enableSuite[GlutenHiveScriptTransformationSuite] + enableSuite[GlutenHiveSerDeReadWriteSuite] + enableSuite[GlutenHiveSerDeSuite] + enableSuite[GlutenHiveTableScanSuite] + enableSuite[GlutenHiveTypeCoercionSuite] + enableSuite[GlutenHiveUDAFSuite] + enableSuite[GlutenHiveUDFSuite] + enableSuite[GlutenObjectHashAggregateSuite] + enableSuite[GlutenPruneHiveTablePartitionsSuite] + enableSuite[GlutenPruningSuite] + enableSuite[GlutenSQLMetricsSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLQuerySuite] + enableSuite[GlutenHashUDAQuerySuite] + enableSuite[GlutenHashUDAQueryWithControlledFallbackSuite] + enableSuite[GlutenSQLQuerySuiteAE] + enableSuite[GlutenWindowQuerySuite] enableSuite[GlutenCollapseProjectExecTransformerSuite] enableSuite[GlutenSparkSessionExtensionSuite] enableSuite[GlutenGroupBasedDeleteFromTableSuite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala new file mode 100644 index 000000000000..e8494d2c075f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashAggregationQuerySuite + extends HashAggregationQuerySuite + with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashAggregationQueryWithControlledFallbackSuite + extends HashAggregationQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala new file mode 100644 index 000000000000..9ce9db428741 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File +import java.nio.file.Files +import java.nio.file.StandardCopyOption + +@SlowHiveTest +class GlutenHiveCommandSuite extends HiveCommandSuite with GlutenTestSetWithSystemPropertyTrait { + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + // Rewritten with a workspace-backed file because TestHive.getHiveFile resolves this resource + // from the spark-hive tests jar in gluten-ut. + "LOAD DATA LOCAL", + "LOAD DATA" + ) + + Seq(true, false).foreach { + local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + testGluten(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = getWorkspaceFilePath( + "sql", + "hive", + "src", + "test", + "resources", + "data", + "files", + "employee.dat").toFile.getCanonicalFile + + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData.toPath, tmp.toPath, StandardCopyOption.REPLACE_EXISTING) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql(""" + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { + path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + sql(""" + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { + f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table")) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala new file mode 100644 index 000000000000..d96078f2d557 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala @@ -0,0 +1,324 @@ +/* + * 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.sql.Dataset +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} + +import java.io.{File, FileOutputStream} +import java.util +import java.util.Locale + +import scala.util.control.NonFatal + +/** + * Mostly copied from Spark's [[HiveComparisonTest]]. This support reuses [[GlutenTestHiveTables]] + * for TestHive table registrations and customizes [[createQueryTest]] so `../../data` resolves to + * workspace-backed Hive test resources. + */ +trait GlutenHiveComparisonTestSupport extends GlutenHiveResourcePathSupport { + this: HiveComparisonTest => + + private val testDataPath: String = { + hiveResourcePath("data").toAbsolutePath.normalize.toUri.getPath + .stripSuffix("/") + } + + private val glutenAnswerCache: File = hiveResourcePath("golden").toFile + + override def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false): Unit = { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logDebug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + + test(testCaseName) { + assume(!skip) + logDebug(s"=== HIVE TEST: $testCaseName ===") + + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // the system to return the wrong answer. Since we have no intention of mirroring their + // previously broken behavior we simply filter out changes to this setting. + .filterNot(_.contains("hive.outerjoin.supports.filters")) + .filterNot(_.contains("hive.exec.post.hooks")) + + if (allQueries != queryList) { + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex + .map { + case (query, i) => + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" + } + .mkString("\n== Console version of this test ==\n", "\n", "\n") + } + + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + if (reset) { + TestHive.reset() + } + + // Register workspace-backed table definitions before lazy TestHive auto-loading kicks in. + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { + query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(glutenAnswerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles + .flatMap { + cachedAnswerFile => + logDebug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(fileToString(cachedAnswerFile)) + } else { + logDebug(s"File $cachedAnswerFile not found") + None + } + } + .map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logInfo(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { + case (queryString, hive) => + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) + } + try { (query, prepareAnswer(query, getResult())) } + catch { + case e: Throwable => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + """.stripMargin + stringToFile( + new File(failedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + queryList.zip(hiveResults).zip(catalystResults).foreach { + case ((query, hive), (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery, hive) + + // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction + if ( + (!hiveQuery.logical.isInstanceOf[ExplainCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && + preparedHive != catalyst + ) { + + val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logWarning(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = + try { + val tablesRead = + new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHiveQueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => + e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead.contains(i.table.identifier) => + (q, e, i) + } + } + + tablesGenerated + .map { + case (hiveql, execution, insert) => + val rdd = + Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${rdd.collect().mkString("\n")} + """.stripMargin + } + .mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + |$computedTablesMessages + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq("into table", "create table", "drop index") + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { + query => excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala new file mode 100644 index 000000000000..b7da63c45a35 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala @@ -0,0 +1,124 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveDDLSuite + extends HiveDDLSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed schema URLs because TestHive.getHiveFile resolves + // these resources from the spark-hive tests jar in gluten-ut. + "SPARK-34370: support Avro schema evolution (add column with avro.schema.url)", + "SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)", + "SPARK-34261: Avoid side effect if create exists temporary function" + ) + + testGluten("SPARK-34370: support Avro schema evolution (add column with avro.schema.url)") { + checkAvroSchemaEvolutionAddColumn( + avroSchemaUrlProperty("schemaWithOneField.avsc"), + avroSchemaUrlProperty("schemaWithTwoFields.avsc")) + } + + testGluten("SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)") { + checkAvroSchemaEvolutionRemoveColumn( + avroSchemaUrlProperty("schemaWithTwoFields.avsc"), + avroSchemaUrlProperty("schemaWithOneField.avsc")) + } + + testGluten("SPARK-34261: Avoid side effect if create exists temporary function") { + withUserDefinedFunction("f1" -> true) { + sql("CREATE TEMPORARY FUNCTION f1 AS 'org.apache.hadoop.hive.ql.udf.UDFUUID'") + + val jarName = "TestUDTF.jar" + val jar = hiveResourcePath(jarName).toUri.toString + spark.sparkContext.addedJars.keys.filter(_.contains(jarName)).foreach { + k => spark.sparkContext.addedJars.remove(k) + } + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + val e = intercept[AnalysisException] { + sql( + "CREATE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + } + checkError( + e, + errorClass = "ROUTINE_ALREADY_EXISTS", + parameters = Map("routineName" -> "`f1`")) + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + + sql( + "CREATE OR REPLACE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + assert(spark.sparkContext.listJars().exists(_.contains(jarName))) + } + } + + private def avroSchemaUrlProperty(fileName: String): String = { + val schemaPath = hiveResourcePath(fileName) + s"'avro.schema.url'='${schemaPath.toUri.toString}'" + } + + private def checkAvroSchemaEvolutionAddColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + checkAnswer( + spark.table("t"), + Row("col1_default", "col2_value", "1981-01-07") :: + Row("col1_value", "col2_value", "1983-04-27") :: Nil) + } + } + + private def checkAvroSchemaEvolutionRemoveColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + checkAnswer( + spark.table("t"), + Row("col2_value", "1981-01-07") :: + Row("col2_value", "1983-04-27") :: Nil) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala new file mode 100644 index 000000000000..0f6b37016691 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveExplainSuite extends HiveExplainSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala new file mode 100644 index 000000000000..6cdf5f5845da --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHivePlanTest extends HivePlanTest with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala new file mode 100644 index 000000000000..f547953c58a3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala @@ -0,0 +1,111 @@ +/* + * 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.SparkFiles +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.HiveUtils.{builtinHiveVersion => hiveVersion} +import org.apache.spark.sql.hive.test.HiveTestJars +import org.apache.spark.tags.SlowHiveTest + +import java.io.File + +@SlowHiveTest +class GlutenHiveQuerySuite + extends HiveQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + "ADD FILE command", + "ADD JAR command 2", + "CREATE TEMPORARY FUNCTION", + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + testGluten("ADD FILE command") { + val testFile = hiveResourcePath("data/files/v1.txt").toFile.toURI + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { + _ => Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + assert(sql("list files").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) + } + + testGluten("ADD JAR command 2") { + val testJar = HiveTestJars.getHiveHcatalogCoreJar().toURI + val testData = hiveResourcePath("data/files/sample.json").toUri + sql(s"ADD JAR $testJar") + withTable("t1") { + sql("""CREATE TABLE t1(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") + sql("select * from src join t1 on src.key = t1.a") + } + assert( + sql("list jars") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + assert( + sql("list jar") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + val testJar2 = hiveResourcePath("TestUDTF.jar").toFile.getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) + } + + testGluten("SPARK-33084: Add jar support Ivy URI in SQL") { + val testData = hiveResourcePath("data/files/sample.json").toUri + withTable("t") { + // Use transitive=false as it should be good enough to test the Ivy support in Hive ADD JAR. + sql( + s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion" + + "?transitive=false") + sql("""CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert( + sql("LIST JARS") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + assert( + sql("LIST JAR") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + } + } + + testGluten("CREATE TEMPORARY FUNCTION") { + val jarURL = hiveResourcePath("TestUDTF.jar").toUri.toURL + sql(s"ADD JAR $jarURL") + withUserDefinedFunction("udtf_count2" -> true) { + sql("""CREATE TEMPORARY FUNCTION udtf_count2 AS + |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |""".stripMargin) + assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala new file mode 100644 index 000000000000..d46c2eed7ad3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveResolutionSuite + extends HiveResolutionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala new file mode 100644 index 000000000000..41a20e8ae6da --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.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.hive.execution + +import org.apache.spark.SparkFunSuite + +import java.nio.file.Path + +/** Shared helpers for resolving workspace-backed Hive test resources. */ +trait GlutenHiveResourcePathSupport { + this: SparkFunSuite => + + protected lazy val hiveTestResourceDir: Path = + getWorkspaceFilePath("sql", "hive", "src", "test", "resources") + + final protected def hiveResourcePath(relativePath: String): Path = { + GlutenTestHiveTables.hiveResourcePath(hiveTestResourceDir, relativePath) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala new file mode 100644 index 000000000000..d537a9aae6f3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSQLViewSuite extends HiveSQLViewSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala new file mode 100644 index 000000000000..158ad4f46e64 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveScriptTransformationSuite + extends HiveScriptTransformationSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala new file mode 100644 index 000000000000..92703dea3711 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeReadWriteSuite + extends HiveSerDeReadWriteSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala new file mode 100644 index 000000000000..3687714a592d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeSuite + extends HiveSerDeSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + /** + * Mostly copied from Spark's [[HiveSerDeSuite]] and [[GlutenTestSetWithSystemPropertyTrait]], and + * customized so `sales.txt` is loaded from the workspace-backed Hive test resources instead of + * via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + + TestHive.setCacheTables(false) + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"""LOAD DATA LOCAL INPATH '${hiveResourcePath("data/files/sales.txt").toFile.toURI}' + |INTO TABLE sales""".stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala new file mode 100644 index 000000000000..ac4903218fcf --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.Row +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTableScanSuite + extends HiveTableScanSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed file because the upstream test resolves this resource + // from the spark-hive tests jar in gluten-ut. + "Spark-4077: timestamp query for null value" + ) + + testGluten("Spark-4077: timestamp query for null value") { + withTable("timestamp_query_null") { + sql(""" + |CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY ',' + |LINES TERMINATED BY '\n' + """.stripMargin) + val location = hiveResourcePath("data/files/issue-4077-data.txt").toFile.toURI + + sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert( + sql("SELECT time FROM timestamp_query_null LIMIT 2").collect() === + Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")), Row(null))) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala new file mode 100644 index 000000000000..d77f2c7d5780 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTypeCoercionSuite + extends HiveTypeCoercionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala new file mode 100644 index 000000000000..975fc9fb8997 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDAFSuite extends HiveUDAFSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala new file mode 100644 index 000000000000..a196da6ab4f4 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDFSuite + extends HiveUDFSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + "UDTF", + "permanent UDTF" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("UDTF") { + withUserDefinedFunction("udtf_count2" -> true) { + sql(s"ADD JAR ${hiveResourcePath("TestUDTF.jar").toFile.getCanonicalPath}") + sql(""" + |CREATE TEMPORARY FUNCTION udtf_count2 + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count2(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } + } + + testGluten("permanent UDTF") { + withUserDefinedFunction("udtf_count_temp" -> false) { + sql(s""" + |CREATE FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count_temp(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..327b633b1f0e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenObjectHashAggregateSuite + extends ObjectHashAggregateSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala new file mode 100644 index 000000000000..14ffda794dcf --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruneHiveTablePartitionsSuite + extends PruneHiveTablePartitionsSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala new file mode 100644 index 000000000000..aeb6e85b6a0c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruningSuite + extends PruningSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala new file mode 100644 index 000000000000..df750c09c49c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenSQLMetricsSuite extends SQLMetricsSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..6a6fa67a6df1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala @@ -0,0 +1,142 @@ +/* + * 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.TestUtils +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME +import org.apache.spark.tags.SlowHiveTest + +import java.sql.Date + +trait GlutenSQLQuerySuiteBase + extends SQLQuerySuiteBase + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed jar paths because TestHive.getHiveFile resolves + // these jars from the Spark test jars in gluten-ut. + "script", + "describe functions - user defined functions", + "describe functions - temporary user defined functions", + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("script") { + withTempView("script_table") { + import spark.implicits._ + + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) + val scriptFilePath = hiveResourcePath("test_script.sh").toFile.getCanonicalPath + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query = sql(s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + } + + testGluten("describe functions - user defined functions") { + withUserDefinedFunction("udtf_count" -> false) { + sql(s""" + |CREATE FUNCTION udtf_count + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: $SESSION_CATALOG_NAME.default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + checkAnswer( + sql("SELECT udtf_count(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: $SESSION_CATALOG_NAME.default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + } + } + + testGluten("describe functions - temporary user defined functions") { + withUserDefinedFunction("udtf_count_temp" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + } + } + + testGluten( + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") { + withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack1 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack2 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack2' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + + Seq("udtf_stack1", "udtf_stack2").foreach { + udf => + checkAnswer( + sql(s"SELECT $udf(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')"), + Seq(Row("A", 10, Date.valueOf("2015-01-01")), Row("B", 20, Date.valueOf("2016-01-01"))) + ) + } + } + } +} + +@SlowHiveTest +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLQuerySuiteBase {} + +@SlowHiveTest +class GlutenSQLQuerySuiteAE extends SQLQuerySuiteAE with GlutenSQLQuerySuiteBase {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala new file mode 100644 index 000000000000..0eaa597a8680 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive.sparkSession +import org.apache.spark.sql.hive.test.TestHive.sparkSession.TestTable +import org.apache.spark.sql.hive.test.TestHiveQueryExecution + +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + +import java.nio.file.Path + +/** Wrappers around [[TestHive]] lazy-loaded table registrations. */ +object GlutenTestHiveTables { + def hiveResourcePath(resourceDir: Path, relativePath: String): Path = { + relativePath.split('/').foldLeft(resourceDir) { case (path, child) => path.resolve(child) } + } + + private def hiveDataFile(resourceDir: Path, fileName: String): String = { + hiveResourcePath(resourceDir, fileName).toAbsolutePath.normalize.toString + } + + implicit private class SqlCmd(sql: String) { + def cmd: () => Unit = { + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit + } + } + + def registerTestTable(testTable: TestTable): Unit = { + sparkSession.registerTestTable(testTable) + } + + def registerHiveQTestUtilsTables(resourceDir: Path): Unit = { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable( + "src", + createTableSQL("src").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |INTO TABLE src""".stripMargin.cmd + ), + TestTable( + "src1", + createTableSQL("src1").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv3.txt")}' + |INTO TABLE src1""".stripMargin.cmd + ), + TestTable( + "srcpart", + () => { + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "srcpart1", + () => { + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "src_thrift", + () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() + + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + } + ), + TestTable( + "serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd + ), + TestTable( + "episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable( + "episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable( + "src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/json.txt")}' + |INTO TABLE src_json""".stripMargin.cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala new file mode 100644 index 000000000000..0d2798995580 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashUDAQuerySuite extends HashUDAQuerySuite with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashUDAQueryWithControlledFallbackSuite + extends HashUDAQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala new file mode 100644 index 000000000000..701fbb8467a9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala @@ -0,0 +1,70 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenWindowQuerySuite + extends WindowQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + /** + * Mostly copied from Spark's [[WindowQuerySuite]] and [[GlutenTestSetWithSystemPropertyTrait]], + * and customized so `part_tiny.txt` is loaded from the workspace-backed Hive test resources + * instead of via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + sql("DROP TABLE IF EXISTS part") + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) USING hive + """.stripMargin) + val testData1 = hiveResourcePath("data/files/part_tiny.txt").toFile.toURI + sql(s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index c908bfe77390..6b5a18a7f170 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.GlutenFallbackSuite -import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.sources._ // Some settings' line length exceeds 100 @@ -821,7 +821,7 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenSerializationSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] - enableSuite[GlutenSQLQuerySuite] + enableSuite[org.apache.spark.sql.GlutenSQLQuerySuite] // Decimal precision exceeds. .exclude("should be able to resolve a persistent view") // Unstable. Needs to be fixed. @@ -861,7 +861,38 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("cases when literal is max") enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHashAggregationQuerySuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHashAggregationQueryWithControlledFallbackSuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHiveCommandSuite] + enableSuite[GlutenHiveDDLSuite] + enableSuite[GlutenHiveExplainSuite] + .exclude("explain output of physical plan should contain proper codegen stage ID") + .exclude("EXPLAIN CODEGEN command") + enableSuite[GlutenHivePlanTest] + enableSuite[GlutenHiveQuerySuite] + enableSuite[GlutenHiveResolutionSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenHiveSQLViewSuite] + enableSuite[GlutenHiveScriptTransformationSuite] + enableSuite[GlutenHiveSerDeReadWriteSuite] + enableSuite[GlutenHiveSerDeSuite] + enableSuite[GlutenHiveTableScanSuite] + enableSuite[GlutenHiveTypeCoercionSuite] + enableSuite[GlutenHiveUDAFSuite] + enableSuite[GlutenHiveUDFSuite] + enableSuite[GlutenObjectHashAggregateSuite] + enableSuite[GlutenPruneHiveTablePartitionsSuite] + enableSuite[GlutenPruningSuite] + enableSuite[GlutenSQLMetricsSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLQuerySuite] + enableSuite[GlutenHashUDAQuerySuite] + enableSuite[GlutenHashUDAQueryWithControlledFallbackSuite] + enableSuite[GlutenSQLQuerySuiteAE] + enableSuite[GlutenWindowQuerySuite] enableSuite[GlutenCollapseProjectExecTransformerSuite] enableSuite[GlutenSparkSessionExtensionSuite] enableSuite[GlutenGroupBasedDeleteFromTableSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala new file mode 100644 index 000000000000..e8494d2c075f --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashAggregationQuerySuite + extends HashAggregationQuerySuite + with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashAggregationQueryWithControlledFallbackSuite + extends HashAggregationQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala new file mode 100644 index 000000000000..9ce9db428741 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File +import java.nio.file.Files +import java.nio.file.StandardCopyOption + +@SlowHiveTest +class GlutenHiveCommandSuite extends HiveCommandSuite with GlutenTestSetWithSystemPropertyTrait { + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + // Rewritten with a workspace-backed file because TestHive.getHiveFile resolves this resource + // from the spark-hive tests jar in gluten-ut. + "LOAD DATA LOCAL", + "LOAD DATA" + ) + + Seq(true, false).foreach { + local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + testGluten(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = getWorkspaceFilePath( + "sql", + "hive", + "src", + "test", + "resources", + "data", + "files", + "employee.dat").toFile.getCanonicalFile + + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData.toPath, tmp.toPath, StandardCopyOption.REPLACE_EXISTING) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql(""" + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { + path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + sql(""" + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { + f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table")) + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala new file mode 100644 index 000000000000..d96078f2d557 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala @@ -0,0 +1,324 @@ +/* + * 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.sql.Dataset +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} + +import java.io.{File, FileOutputStream} +import java.util +import java.util.Locale + +import scala.util.control.NonFatal + +/** + * Mostly copied from Spark's [[HiveComparisonTest]]. This support reuses [[GlutenTestHiveTables]] + * for TestHive table registrations and customizes [[createQueryTest]] so `../../data` resolves to + * workspace-backed Hive test resources. + */ +trait GlutenHiveComparisonTestSupport extends GlutenHiveResourcePathSupport { + this: HiveComparisonTest => + + private val testDataPath: String = { + hiveResourcePath("data").toAbsolutePath.normalize.toUri.getPath + .stripSuffix("/") + } + + private val glutenAnswerCache: File = hiveResourcePath("golden").toFile + + override def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false): Unit = { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logDebug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + + test(testCaseName) { + assume(!skip) + logDebug(s"=== HIVE TEST: $testCaseName ===") + + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // the system to return the wrong answer. Since we have no intention of mirroring their + // previously broken behavior we simply filter out changes to this setting. + .filterNot(_.contains("hive.outerjoin.supports.filters")) + .filterNot(_.contains("hive.exec.post.hooks")) + + if (allQueries != queryList) { + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex + .map { + case (query, i) => + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" + } + .mkString("\n== Console version of this test ==\n", "\n", "\n") + } + + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + if (reset) { + TestHive.reset() + } + + // Register workspace-backed table definitions before lazy TestHive auto-loading kicks in. + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { + query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(glutenAnswerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles + .flatMap { + cachedAnswerFile => + logDebug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(fileToString(cachedAnswerFile)) + } else { + logDebug(s"File $cachedAnswerFile not found") + None + } + } + .map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logInfo(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { + case (queryString, hive) => + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) + } + try { (query, prepareAnswer(query, getResult())) } + catch { + case e: Throwable => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + """.stripMargin + stringToFile( + new File(failedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + queryList.zip(hiveResults).zip(catalystResults).foreach { + case ((query, hive), (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery, hive) + + // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction + if ( + (!hiveQuery.logical.isInstanceOf[ExplainCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && + preparedHive != catalyst + ) { + + val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logWarning(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = + try { + val tablesRead = + new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHiveQueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => + e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead.contains(i.table.identifier) => + (q, e, i) + } + } + + tablesGenerated + .map { + case (hiveql, execution, insert) => + val rdd = + Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${rdd.collect().mkString("\n")} + """.stripMargin + } + .mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + |$computedTablesMessages + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq("into table", "create table", "drop index") + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { + query => excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) + } + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala new file mode 100644 index 000000000000..96262d2ff9b6 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala @@ -0,0 +1,124 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveDDLSuite + extends HiveDDLSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed schema URLs because TestHive.getHiveFile resolves + // these resources from the spark-hive tests jar in gluten-ut. + "SPARK-34370: support Avro schema evolution (add column with avro.schema.url)", + "SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)", + "SPARK-34261: Avoid side effect if create exists temporary function" + ) + + testGluten("SPARK-34370: support Avro schema evolution (add column with avro.schema.url)") { + checkAvroSchemaEvolutionAddColumn( + avroSchemaUrlProperty("schemaWithOneField.avsc"), + avroSchemaUrlProperty("schemaWithTwoFields.avsc")) + } + + testGluten("SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)") { + checkAvroSchemaEvolutionRemoveColumn( + avroSchemaUrlProperty("schemaWithTwoFields.avsc"), + avroSchemaUrlProperty("schemaWithOneField.avsc")) + } + + testGluten("SPARK-34261: Avoid side effect if create exists temporary function") { + withUserDefinedFunction("f1" -> true) { + sql("CREATE TEMPORARY FUNCTION f1 AS 'org.apache.hadoop.hive.ql.udf.UDFUUID'") + + val jarName = "TestUDTF.jar" + val jar = hiveResourcePath(jarName).toUri.toString + spark.sparkContext.allAddedJars.keys.filter(_.contains(jarName)).foreach { + k => spark.sparkContext.addedJars.values.foreach(_.remove(k)) + } + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + val e = intercept[AnalysisException] { + sql( + "CREATE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + } + checkError( + e, + errorClass = "ROUTINE_ALREADY_EXISTS", + parameters = Map("routineName" -> "`f1`")) + assert(!spark.sparkContext.listJars().exists(_.contains(jarName))) + + sql( + "CREATE OR REPLACE TEMPORARY FUNCTION f1 AS " + + s"'org.apache.hadoop.hive.ql.udf.UDFUUID' USING JAR '$jar'") + assert(spark.sparkContext.listJars().exists(_.contains(jarName))) + } + } + + private def avroSchemaUrlProperty(fileName: String): String = { + val schemaPath = hiveResourcePath(fileName) + s"'avro.schema.url'='${schemaPath.toUri.toString}'" + } + + private def checkAvroSchemaEvolutionAddColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + checkAnswer( + spark.table("t"), + Row("col1_default", "col2_value", "1981-01-07") :: + Row("col1_value", "col2_value", "1983-04-27") :: Nil) + } + } + + private def checkAvroSchemaEvolutionRemoveColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + checkAnswer( + spark.table("t"), + Row("col2_value", "1981-01-07") :: + Row("col2_value", "1983-04-27") :: Nil) + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala new file mode 100644 index 000000000000..0f6b37016691 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveExplainSuite extends HiveExplainSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala new file mode 100644 index 000000000000..6cdf5f5845da --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHivePlanTest extends HivePlanTest with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala new file mode 100644 index 000000000000..2b4c715a513b --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala @@ -0,0 +1,110 @@ +/* + * 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.SparkFiles +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.HiveUtils.{builtinHiveVersion => hiveVersion} +import org.apache.spark.sql.hive.test.HiveTestJars +import org.apache.spark.tags.SlowHiveTest + +import java.io.File + +@SlowHiveTest +class GlutenHiveQuerySuite + extends HiveQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + "ADD FILE command", + "ADD JAR command 2", + "CREATE TEMPORARY FUNCTION", + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + testGluten("ADD FILE command") { + val testFile = hiveResourcePath("data/files/v1.txt").toFile.toURI + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { + _ => Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + assert(sql("list files").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) + } + + testGluten("ADD JAR command 2") { + val testJar = HiveTestJars.getHiveHcatalogCoreJar().toURI + val testData = hiveResourcePath("data/files/sample.json").toUri + sql(s"ADD JAR $testJar") + withTable("t1") { + sql("""CREATE TABLE t1(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") + sql("select * from src join t1 on src.key = t1.a") + } + assert( + sql("list jars") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + assert( + sql("list jar") + .filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)) + .count() > 0) + val testJar2 = hiveResourcePath("TestUDTF.jar").toFile.getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) + } + + testGluten("SPARK-33084: Add jar support Ivy URI in SQL") { + val testData = hiveResourcePath("data/files/sample.json").toUri + withTable("t") { + // Use transitive=false as it should be good enough to test the Ivy support in Hive ADD JAR. + sql( + s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion?transitive=false") + sql("""CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert( + sql("LIST JARS") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + assert( + sql("LIST JAR") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + } + } + + testGluten("CREATE TEMPORARY FUNCTION") { + val jarURL = hiveResourcePath("TestUDTF.jar").toUri.toURL + sql(s"ADD JAR $jarURL") + withUserDefinedFunction("udtf_count2" -> true) { + sql("""CREATE TEMPORARY FUNCTION udtf_count2 AS + |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |""".stripMargin) + assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1) + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala new file mode 100644 index 000000000000..d46c2eed7ad3 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveResolutionSuite + extends HiveResolutionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala new file mode 100644 index 000000000000..41a20e8ae6da --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.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.hive.execution + +import org.apache.spark.SparkFunSuite + +import java.nio.file.Path + +/** Shared helpers for resolving workspace-backed Hive test resources. */ +trait GlutenHiveResourcePathSupport { + this: SparkFunSuite => + + protected lazy val hiveTestResourceDir: Path = + getWorkspaceFilePath("sql", "hive", "src", "test", "resources") + + final protected def hiveResourcePath(relativePath: String): Path = { + GlutenTestHiveTables.hiveResourcePath(hiveTestResourceDir, relativePath) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala new file mode 100644 index 000000000000..d537a9aae6f3 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSQLViewSuite extends HiveSQLViewSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala new file mode 100644 index 000000000000..158ad4f46e64 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveScriptTransformationSuite + extends HiveScriptTransformationSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala new file mode 100644 index 000000000000..92703dea3711 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeReadWriteSuite + extends HiveSerDeReadWriteSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala new file mode 100644 index 000000000000..3687714a592d --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeSuite + extends HiveSerDeSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + /** + * Mostly copied from Spark's [[HiveSerDeSuite]] and [[GlutenTestSetWithSystemPropertyTrait]], and + * customized so `sales.txt` is loaded from the workspace-backed Hive test resources instead of + * via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + + TestHive.setCacheTables(false) + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"""LOAD DATA LOCAL INPATH '${hiveResourcePath("data/files/sales.txt").toFile.toURI}' + |INTO TABLE sales""".stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala new file mode 100644 index 000000000000..ac4903218fcf --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.Row +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTableScanSuite + extends HiveTableScanSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed file because the upstream test resolves this resource + // from the spark-hive tests jar in gluten-ut. + "Spark-4077: timestamp query for null value" + ) + + testGluten("Spark-4077: timestamp query for null value") { + withTable("timestamp_query_null") { + sql(""" + |CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY ',' + |LINES TERMINATED BY '\n' + """.stripMargin) + val location = hiveResourcePath("data/files/issue-4077-data.txt").toFile.toURI + + sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert( + sql("SELECT time FROM timestamp_query_null LIMIT 2").collect() === + Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")), Row(null))) + } + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala new file mode 100644 index 000000000000..d77f2c7d5780 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTypeCoercionSuite + extends HiveTypeCoercionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala new file mode 100644 index 000000000000..975fc9fb8997 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDAFSuite extends HiveUDAFSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala new file mode 100644 index 000000000000..6a5ccdb625ed --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -0,0 +1,32 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDFSuite + extends HiveUDFSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..327b633b1f0e --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenObjectHashAggregateSuite + extends ObjectHashAggregateSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala new file mode 100644 index 000000000000..14ffda794dcf --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruneHiveTablePartitionsSuite + extends PruneHiveTablePartitionsSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala new file mode 100644 index 000000000000..aeb6e85b6a0c --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruningSuite + extends PruningSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala new file mode 100644 index 000000000000..df750c09c49c --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenSQLMetricsSuite extends SQLMetricsSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..6a6fa67a6df1 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala @@ -0,0 +1,142 @@ +/* + * 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.TestUtils +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME +import org.apache.spark.tags.SlowHiveTest + +import java.sql.Date + +trait GlutenSQLQuerySuiteBase + extends SQLQuerySuiteBase + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed jar paths because TestHive.getHiveFile resolves + // these jars from the Spark test jars in gluten-ut. + "script", + "describe functions - user defined functions", + "describe functions - temporary user defined functions", + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("script") { + withTempView("script_table") { + import spark.implicits._ + + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) + val scriptFilePath = hiveResourcePath("test_script.sh").toFile.getCanonicalPath + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query = sql(s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + } + + testGluten("describe functions - user defined functions") { + withUserDefinedFunction("udtf_count" -> false) { + sql(s""" + |CREATE FUNCTION udtf_count + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: $SESSION_CATALOG_NAME.default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + checkAnswer( + sql("SELECT udtf_count(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count"), + s"Function: $SESSION_CATALOG_NAME.default.udtf_count", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A" + ) + } + } + + testGluten("describe functions - temporary user defined functions") { + withUserDefinedFunction("udtf_count_temp" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveResourcePath("TestUDTF.jar").toUri}' + """.stripMargin) + + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + checkKeywordsExist( + sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + s"Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + } + } + + testGluten( + "SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") { + withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack1 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + sql(s""" + |CREATE TEMPORARY FUNCTION udtf_stack2 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack2' + |USING JAR '${hiveResourcePath("SPARK-21101-1.0.jar").toUri}' + """.stripMargin) + + Seq("udtf_stack1", "udtf_stack2").foreach { + udf => + checkAnswer( + sql(s"SELECT $udf(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')"), + Seq(Row("A", 10, Date.valueOf("2015-01-01")), Row("B", 20, Date.valueOf("2016-01-01"))) + ) + } + } + } +} + +@SlowHiveTest +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLQuerySuiteBase {} + +@SlowHiveTest +class GlutenSQLQuerySuiteAE extends SQLQuerySuiteAE with GlutenSQLQuerySuiteBase {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala new file mode 100644 index 000000000000..0eaa597a8680 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive.sparkSession +import org.apache.spark.sql.hive.test.TestHive.sparkSession.TestTable +import org.apache.spark.sql.hive.test.TestHiveQueryExecution + +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + +import java.nio.file.Path + +/** Wrappers around [[TestHive]] lazy-loaded table registrations. */ +object GlutenTestHiveTables { + def hiveResourcePath(resourceDir: Path, relativePath: String): Path = { + relativePath.split('/').foldLeft(resourceDir) { case (path, child) => path.resolve(child) } + } + + private def hiveDataFile(resourceDir: Path, fileName: String): String = { + hiveResourcePath(resourceDir, fileName).toAbsolutePath.normalize.toString + } + + implicit private class SqlCmd(sql: String) { + def cmd: () => Unit = { + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit + } + } + + def registerTestTable(testTable: TestTable): Unit = { + sparkSession.registerTestTable(testTable) + } + + def registerHiveQTestUtilsTables(resourceDir: Path): Unit = { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable( + "src", + createTableSQL("src").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |INTO TABLE src""".stripMargin.cmd + ), + TestTable( + "src1", + createTableSQL("src1").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv3.txt")}' + |INTO TABLE src1""".stripMargin.cmd + ), + TestTable( + "srcpart", + () => { + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "srcpart1", + () => { + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "src_thrift", + () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() + + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + } + ), + TestTable( + "serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd + ), + TestTable( + "episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable( + "episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable( + "src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/json.txt")}' + |INTO TABLE src_json""".stripMargin.cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala new file mode 100644 index 000000000000..0d2798995580 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashUDAQuerySuite extends HashUDAQuerySuite with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashUDAQueryWithControlledFallbackSuite + extends HashUDAQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala new file mode 100644 index 000000000000..701fbb8467a9 --- /dev/null +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala @@ -0,0 +1,70 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenWindowQuerySuite + extends WindowQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + /** + * Mostly copied from Spark's [[WindowQuerySuite]] and [[GlutenTestSetWithSystemPropertyTrait]], + * and customized so `part_tiny.txt` is loaded from the workspace-backed Hive test resources + * instead of via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + sql("DROP TABLE IF EXISTS part") + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) USING hive + """.stripMargin) + val testData1 = hiveResourcePath("data/files/part_tiny.txt").toFile.toURI + sql(s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} 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 index f639f9c39c15..a63b260af9b9 100644 --- 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 @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution.metric.{GlutenCustomMetricsSuite, GlutenSQ import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.{GlutenFallbackStrategiesSuite, GlutenFallbackSuite} -import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming._ @@ -1042,7 +1042,7 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenSerializationSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] - enableSuite[GlutenSQLQuerySuite] + enableSuite[org.apache.spark.sql.GlutenSQLQuerySuite] // Decimal precision exceeds. .exclude("should be able to resolve a persistent view") // Unstable. Needs to be fixed. @@ -1089,7 +1089,38 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHashAggregationQuerySuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHashAggregationQueryWithControlledFallbackSuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHiveCommandSuite] + enableSuite[GlutenHiveDDLSuite] + enableSuite[GlutenHiveExplainSuite] + .exclude("explain output of physical plan should contain proper codegen stage ID") + .exclude("EXPLAIN CODEGEN command") + enableSuite[GlutenHivePlanTest] + enableSuite[GlutenHiveQuerySuite] + enableSuite[GlutenHiveResolutionSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenHiveSQLViewSuite] + enableSuite[GlutenHiveScriptTransformationSuite] + enableSuite[GlutenHiveSerDeReadWriteSuite] + enableSuite[GlutenHiveSerDeSuite] + enableSuite[GlutenHiveTableScanSuite] + enableSuite[GlutenHiveTypeCoercionSuite] + enableSuite[GlutenHiveUDAFSuite] + enableSuite[GlutenHiveUDFSuite] + enableSuite[GlutenObjectHashAggregateSuite] + enableSuite[GlutenPruneHiveTablePartitionsSuite] + enableSuite[GlutenPruningSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLMetricsSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLQuerySuite] + enableSuite[GlutenHashUDAQuerySuite] + enableSuite[GlutenHashUDAQueryWithControlledFallbackSuite] + enableSuite[GlutenSQLQuerySuiteAE] + enableSuite[GlutenWindowQuerySuite] enableSuite[GlutenCollapseProjectExecTransformerSuite] // TODO: 4.x enableSuite[GlutenSparkSessionExtensionSuite] // 1 failure enableSuite[GlutenGroupBasedDeleteFromTableSuite] diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala new file mode 100644 index 000000000000..e8494d2c075f --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashAggregationQuerySuite + extends HashAggregationQuerySuite + with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashAggregationQueryWithControlledFallbackSuite + extends HashAggregationQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala new file mode 100644 index 000000000000..9ce9db428741 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File +import java.nio.file.Files +import java.nio.file.StandardCopyOption + +@SlowHiveTest +class GlutenHiveCommandSuite extends HiveCommandSuite with GlutenTestSetWithSystemPropertyTrait { + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + // Rewritten with a workspace-backed file because TestHive.getHiveFile resolves this resource + // from the spark-hive tests jar in gluten-ut. + "LOAD DATA LOCAL", + "LOAD DATA" + ) + + Seq(true, false).foreach { + local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + testGluten(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = getWorkspaceFilePath( + "sql", + "hive", + "src", + "test", + "resources", + "data", + "files", + "employee.dat").toFile.getCanonicalFile + + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData.toPath, tmp.toPath, StandardCopyOption.REPLACE_EXISTING) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql(""" + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { + path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + sql(""" + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { + f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table")) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala new file mode 100644 index 000000000000..3850a81225f2 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala @@ -0,0 +1,323 @@ +/* + * 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.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.classic.Dataset +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} + +import java.io.{File, FileOutputStream} +import java.nio.file.Files +import java.util +import java.util.Locale + +import scala.util.control.NonFatal + +/** + * Mostly copied from Spark's [[HiveComparisonTest]]. This support reuses [[GlutenTestHiveTables]] + * for TestHive table registrations and customizes [[createQueryTest]] so `../../data` resolves to + * workspace-backed Hive test resources. + */ +trait GlutenHiveComparisonTestSupport extends GlutenHiveResourcePathSupport { + this: HiveComparisonTest => + + private val testDataPath: String = { + hiveResourcePath("data").toAbsolutePath.normalize.toUri.getPath + .stripSuffix("/") + } + + override def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false): Unit = { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logDebug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + + test(testCaseName) { + assume(!skip) + logDebug(s"=== HIVE TEST: $testCaseName ===") + + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // the system to return the wrong answer. Since we have no intention of mirroring their + // previously broken behavior we simply filter out changes to this setting. + .filterNot(_.contains("hive.outerjoin.supports.filters")) + .filterNot(_.contains("hive.exec.post.hooks")) + + if (allQueries != queryList) { + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex + .map { + case (query, i) => + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" + } + .mkString("\n== Console version of this test ==\n", "\n", "\n") + } + + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + if (reset) { + TestHive.reset() + } + + // Register workspace-backed table definitions before lazy TestHive auto-loading kicks in. + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { + query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(answerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles + .flatMap { + cachedAnswerFile => + logDebug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(Files.readString(cachedAnswerFile.toPath)) + } else { + logDebug(s"File $cachedAnswerFile not found") + None + } + } + .map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logInfo(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { + case (queryString, hive) => + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) + } + try { (query, prepareAnswer(query, getResult())) } + catch { + case e: Throwable => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + """.stripMargin + stringToFile( + new File(failedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + queryList.lazyZip(hiveResults).lazyZip(catalystResults).foreach { + case (query, hive, (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery, hive) + + // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction + if ( + (!hiveQuery.logical.isInstanceOf[ExplainCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && + preparedHive != catalyst + ) { + + val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logWarning(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = + try { + val tablesRead = + new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHiveQueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => + e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead.contains(i.table.identifier) => + (q, e, i) + } + } + + tablesGenerated + .map { + case (hiveql, execution, insert) => + val rdd = + Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${rdd.collect().mkString("\n")} + """.stripMargin + } + .mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + |$computedTablesMessages + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq("into table", "create table", "drop index") + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { + query => excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) + } + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala new file mode 100644 index 000000000000..10432173b776 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.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.hive.execution + +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveDDLSuite + extends HiveDDLSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed schema URLs because TestHive.getHiveFile resolves + // these resources from the spark-hive tests jar in gluten-ut. + "SPARK-34370: support Avro schema evolution (add column with avro.schema.url)", + "SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)" + ) + + testGluten("SPARK-34370: support Avro schema evolution (add column with avro.schema.url)") { + checkAvroSchemaEvolutionAddColumn( + avroSchemaUrlProperty("schemaWithOneField.avsc"), + avroSchemaUrlProperty("schemaWithTwoFields.avsc")) + } + + testGluten("SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)") { + checkAvroSchemaEvolutionRemoveColumn( + avroSchemaUrlProperty("schemaWithTwoFields.avsc"), + avroSchemaUrlProperty("schemaWithOneField.avsc")) + } + + private def avroSchemaUrlProperty(fileName: String): String = { + val schemaPath = hiveResourcePath(fileName) + s"'avro.schema.url'='${schemaPath.toUri.toString}'" + } + + private def checkAvroSchemaEvolutionAddColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + checkAnswer( + spark.table("t"), + Row("col1_default", "col2_value", "1981-01-07") :: + Row("col1_value", "col2_value", "1983-04-27") :: Nil) + } + } + + private def checkAvroSchemaEvolutionRemoveColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + checkAnswer( + spark.table("t"), + Row("col2_value", "1981-01-07") :: + Row("col2_value", "1983-04-27") :: Nil) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala new file mode 100644 index 000000000000..0f6b37016691 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveExplainSuite extends HiveExplainSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala new file mode 100644 index 000000000000..6cdf5f5845da --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHivePlanTest extends HivePlanTest with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala new file mode 100644 index 000000000000..4b72a089864a --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala @@ -0,0 +1,73 @@ +/* + * 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.SparkFiles +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.HiveUtils.{builtinHiveVersion => hiveVersion} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File + +@SlowHiveTest +class GlutenHiveQuerySuite + extends HiveQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + "ADD FILE command", + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + testGluten("ADD FILE command") { + val testFile = hiveResourcePath("data/files/v1.txt").toFile.toURI + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { + _ => Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + assert(sql("list files").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) + } + + testGluten("SPARK-33084: Add jar support Ivy URI in SQL") { + val testData = hiveResourcePath("data/files/sample.json").toUri + withTable("t") { + // Use transitive=false as it should be good enough to test the Ivy support in Hive ADD JAR. + sql( + s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion?transitive=false") + sql("""CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert( + sql("LIST JARS") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + assert( + sql("LIST JAR") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala new file mode 100644 index 000000000000..d46c2eed7ad3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveResolutionSuite + extends HiveResolutionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala new file mode 100644 index 000000000000..41a20e8ae6da --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.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.hive.execution + +import org.apache.spark.SparkFunSuite + +import java.nio.file.Path + +/** Shared helpers for resolving workspace-backed Hive test resources. */ +trait GlutenHiveResourcePathSupport { + this: SparkFunSuite => + + protected lazy val hiveTestResourceDir: Path = + getWorkspaceFilePath("sql", "hive", "src", "test", "resources") + + final protected def hiveResourcePath(relativePath: String): Path = { + GlutenTestHiveTables.hiveResourcePath(hiveTestResourceDir, relativePath) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala new file mode 100644 index 000000000000..d537a9aae6f3 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSQLViewSuite extends HiveSQLViewSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala new file mode 100644 index 000000000000..158ad4f46e64 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveScriptTransformationSuite + extends HiveScriptTransformationSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala new file mode 100644 index 000000000000..92703dea3711 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeReadWriteSuite + extends HiveSerDeReadWriteSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala new file mode 100644 index 000000000000..3687714a592d --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeSuite + extends HiveSerDeSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + /** + * Mostly copied from Spark's [[HiveSerDeSuite]] and [[GlutenTestSetWithSystemPropertyTrait]], and + * customized so `sales.txt` is loaded from the workspace-backed Hive test resources instead of + * via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + + TestHive.setCacheTables(false) + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"""LOAD DATA LOCAL INPATH '${hiveResourcePath("data/files/sales.txt").toFile.toURI}' + |INTO TABLE sales""".stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala new file mode 100644 index 000000000000..ac4903218fcf --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.Row +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTableScanSuite + extends HiveTableScanSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed file because the upstream test resolves this resource + // from the spark-hive tests jar in gluten-ut. + "Spark-4077: timestamp query for null value" + ) + + testGluten("Spark-4077: timestamp query for null value") { + withTable("timestamp_query_null") { + sql(""" + |CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY ',' + |LINES TERMINATED BY '\n' + """.stripMargin) + val location = hiveResourcePath("data/files/issue-4077-data.txt").toFile.toURI + + sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert( + sql("SELECT time FROM timestamp_query_null LIMIT 2").collect() === + Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")), Row(null))) + } + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala new file mode 100644 index 000000000000..d77f2c7d5780 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTypeCoercionSuite + extends HiveTypeCoercionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala new file mode 100644 index 000000000000..975fc9fb8997 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDAFSuite extends HiveUDAFSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala new file mode 100644 index 000000000000..6a5ccdb625ed --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -0,0 +1,32 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDFSuite + extends HiveUDFSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..327b633b1f0e --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenObjectHashAggregateSuite + extends ObjectHashAggregateSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala new file mode 100644 index 000000000000..14ffda794dcf --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruneHiveTablePartitionsSuite + extends PruneHiveTablePartitionsSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala new file mode 100644 index 000000000000..aeb6e85b6a0c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruningSuite + extends PruningSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala new file mode 100644 index 000000000000..df750c09c49c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenSQLMetricsSuite extends SQLMetricsSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..9af7bfca463c --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.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.hive.execution + +import org.apache.spark.TestUtils +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +trait GlutenSQLQuerySuiteBase + extends SQLQuerySuiteBase + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed script path because getTestResourcePath resolves + // test-script.sh from the spark-hive tests jar in gluten-ut. + "script" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("script") { + withTempView("script_table") { + import spark.implicits._ + + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) + val scriptFilePath = hiveResourcePath("test-script.sh").toFile.getCanonicalPath + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query = sql(s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + } +} + +@SlowHiveTest +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLQuerySuiteBase {} + +@SlowHiveTest +class GlutenSQLQuerySuiteAE extends SQLQuerySuiteAE with GlutenSQLQuerySuiteBase {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala new file mode 100644 index 000000000000..0eaa597a8680 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive.sparkSession +import org.apache.spark.sql.hive.test.TestHive.sparkSession.TestTable +import org.apache.spark.sql.hive.test.TestHiveQueryExecution + +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + +import java.nio.file.Path + +/** Wrappers around [[TestHive]] lazy-loaded table registrations. */ +object GlutenTestHiveTables { + def hiveResourcePath(resourceDir: Path, relativePath: String): Path = { + relativePath.split('/').foldLeft(resourceDir) { case (path, child) => path.resolve(child) } + } + + private def hiveDataFile(resourceDir: Path, fileName: String): String = { + hiveResourcePath(resourceDir, fileName).toAbsolutePath.normalize.toString + } + + implicit private class SqlCmd(sql: String) { + def cmd: () => Unit = { + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit + } + } + + def registerTestTable(testTable: TestTable): Unit = { + sparkSession.registerTestTable(testTable) + } + + def registerHiveQTestUtilsTables(resourceDir: Path): Unit = { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable( + "src", + createTableSQL("src").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |INTO TABLE src""".stripMargin.cmd + ), + TestTable( + "src1", + createTableSQL("src1").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv3.txt")}' + |INTO TABLE src1""".stripMargin.cmd + ), + TestTable( + "srcpart", + () => { + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "srcpart1", + () => { + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "src_thrift", + () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() + + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + } + ), + TestTable( + "serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd + ), + TestTable( + "episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable( + "episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable( + "src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/json.txt")}' + |INTO TABLE src_json""".stripMargin.cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + } +} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala new file mode 100644 index 000000000000..0d2798995580 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashUDAQuerySuite extends HashUDAQuerySuite with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashUDAQueryWithControlledFallbackSuite + extends HashUDAQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala new file mode 100644 index 000000000000..701fbb8467a9 --- /dev/null +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala @@ -0,0 +1,70 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenWindowQuerySuite + extends WindowQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + /** + * Mostly copied from Spark's [[WindowQuerySuite]] and [[GlutenTestSetWithSystemPropertyTrait]], + * and customized so `part_tiny.txt` is loaded from the workspace-backed Hive test resources + * instead of via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + sql("DROP TABLE IF EXISTS part") + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) USING hive + """.stripMargin) + val testData1 = hiveResourcePath("data/files/part_tiny.txt").toFile.toURI + sql(s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 71b12e340e25..fd521c92f036 100644 --- a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution.metric.{GlutenCustomMetricsSuite, GlutenSQ import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.{GlutenFallbackStrategiesSuite, GlutenFallbackSuite} -import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming._ @@ -1042,7 +1042,7 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenSerializationSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] - enableSuite[GlutenSQLQuerySuite] + enableSuite[org.apache.spark.sql.GlutenSQLQuerySuite] // Decimal precision exceeds. .exclude("should be able to resolve a persistent view") // Unstable. Needs to be fixed. @@ -1093,7 +1093,38 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenUnsafeRowChecksumSuite] enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHashAggregationQuerySuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHashAggregationQueryWithControlledFallbackSuite] + // TODO: fix on https://github.com/apache/gluten/issues/11919 + .exclude("udaf with all data types") + enableSuite[GlutenHiveCommandSuite] + enableSuite[GlutenHiveDDLSuite] + enableSuite[GlutenHiveExplainSuite] + .exclude("explain output of physical plan should contain proper codegen stage ID") + .exclude("EXPLAIN CODEGEN command") + enableSuite[GlutenHivePlanTest] + enableSuite[GlutenHiveQuerySuite] + enableSuite[GlutenHiveResolutionSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenHiveSQLViewSuite] + enableSuite[GlutenHiveScriptTransformationSuite] + enableSuite[GlutenHiveSerDeReadWriteSuite] + enableSuite[GlutenHiveSerDeSuite] + enableSuite[GlutenHiveTableScanSuite] + enableSuite[GlutenHiveTypeCoercionSuite] + enableSuite[GlutenHiveUDAFSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenHiveUDFSuite] + enableSuite[GlutenObjectHashAggregateSuite] + enableSuite[GlutenPruneHiveTablePartitionsSuite] + enableSuite[GlutenPruningSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLMetricsSuite] + enableSuite[org.apache.spark.sql.hive.execution.GlutenSQLQuerySuite] + enableSuite[GlutenHashUDAQuerySuite] + enableSuite[GlutenHashUDAQueryWithControlledFallbackSuite] + enableSuite[GlutenSQLQuerySuiteAE] + enableSuite[GlutenWindowQuerySuite] enableSuite[GlutenCollapseProjectExecTransformerSuite] // TODO: 4.x enableSuite[GlutenSparkSessionExtensionSuite] // 1 failure enableSuite[GlutenGroupBasedDeleteFromTableSuite] diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.scala new file mode 100644 index 000000000000..e8494d2c075f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenAggregationQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashAggregationQuerySuite + extends HashAggregationQuerySuite + with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashAggregationQueryWithControlledFallbackSuite + extends HashAggregationQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala new file mode 100644 index 000000000000..9ce9db428741 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveCommandSuite.scala @@ -0,0 +1,171 @@ +/* + * 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.sql.{AnalysisException, GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File +import java.nio.file.Files +import java.nio.file.StandardCopyOption + +@SlowHiveTest +class GlutenHiveCommandSuite extends HiveCommandSuite with GlutenTestSetWithSystemPropertyTrait { + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + // Rewritten with a workspace-backed file because TestHive.getHiveFile resolves this resource + // from the spark-hive tests jar in gluten-ut. + "LOAD DATA LOCAL", + "LOAD DATA" + ) + + Seq(true, false).foreach { + local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + testGluten(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = getWorkspaceFilePath( + "sql", + "hive", + "src", + "test", + "resources", + "data", + "files", + "employee.dat").toFile.getCanonicalFile + + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData.toPath, tmp.toPath, StandardCopyOption.REPLACE_EXISTING) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql(""" + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { + path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { + path => sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer(sql("SELECT * FROM non_part_table WHERE employeeID = 16"), Row(16, "john") :: Nil) + + sql(""" + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { + f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { + f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table")) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala new file mode 100644 index 000000000000..3850a81225f2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveComparisonTestSupport.scala @@ -0,0 +1,323 @@ +/* + * 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.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.classic.Dataset +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} + +import java.io.{File, FileOutputStream} +import java.nio.file.Files +import java.util +import java.util.Locale + +import scala.util.control.NonFatal + +/** + * Mostly copied from Spark's [[HiveComparisonTest]]. This support reuses [[GlutenTestHiveTables]] + * for TestHive table registrations and customizes [[createQueryTest]] so `../../data` resolves to + * workspace-backed Hive test resources. + */ +trait GlutenHiveComparisonTestSupport extends GlutenHiveResourcePathSupport { + this: HiveComparisonTest => + + private val testDataPath: String = { + hiveResourcePath("data").toAbsolutePath.normalize.toUri.getPath + .stripSuffix("/") + } + + override def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false): Unit = { + // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows + assert(!testCaseName.contains(":")) + + // If test sharding is enable, skip tests that are not in the correct shard. + shardInfo.foreach { + case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") + } + + // Skip tests found in directories specified by user. + skipDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + .foreach(_ => return) + + // If runonlytests is set, skip this test unless we find a file in one of the specified + // directories. + val runIndicators = + runOnlyDirectories + .map(new File(_, testCaseName)) + .filter(_.exists) + if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { + logDebug( + s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") + return + } + + test(testCaseName) { + assume(!skip) + logDebug(s"=== HIVE TEST: $testCaseName ===") + + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + + // TODO: DOCUMENT UNSUPPORTED + val queryList = + allQueries + // In hive, setting the hive.outerjoin.supports.filters flag to "false" essentially tells + // the system to return the wrong answer. Since we have no intention of mirroring their + // previously broken behavior we simply filter out changes to this setting. + .filterNot(_.contains("hive.outerjoin.supports.filters")) + .filterNot(_.contains("hive.exec.post.hooks")) + + if (allQueries != queryList) { + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } + + lazy val consoleTestCase = { + val quotes = "\"\"\"" + queryList.zipWithIndex + .map { + case (query, i) => + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" + } + .mkString("\n== Console version of this test ==\n", "\n", "\n") + } + + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + + if (reset) { + TestHive.reset() + } + + // Register workspace-backed table definitions before lazy TestHive auto-loading kicks in. + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { + query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(answerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles + .flatMap { + cachedAnswerFile => + logDebug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(Files.readString(cachedAnswerFile.toPath)) + } else { + logDebug(s"File $cachedAnswerFile not found") + None + } + } + .map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logInfo(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { + case (queryString, hive) => + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) + } + try { (query, prepareAnswer(query, getResult())) } + catch { + case e: Throwable => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$queryString + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + """.stripMargin + stringToFile( + new File(failedDirectory, testCaseName), + errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + queryList.lazyZip(hiveResults).lazyZip(catalystResults).foreach { + case (query, hive, (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery, hive) + + // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction + if ( + (!hiveQuery.logical.isInstanceOf[ExplainCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && + preparedHive != catalyst + ) { + + val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive + val catalystPrintOut = s"== CATALYST - ${catalyst.size} row(s) ==" +: catalyst + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logWarning(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = + try { + val tablesRead = + new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHiveQueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => + e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead.contains(i.table.identifier) => + (q, e, i) + } + } + + tablesGenerated + .map { + case (hiveql, execution, insert) => + val rdd = + Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${rdd.collect().mkString("\n")} + """.stripMargin + } + .mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + |$computedTablesMessages + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq("into table", "create table", "drop index") + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { + query => excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.scala new file mode 100644 index 000000000000..10432173b776 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveDDLSuite.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.hive.execution + +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveDDLSuite + extends HiveDDLSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with workspace-backed schema URLs because TestHive.getHiveFile resolves + // these resources from the spark-hive tests jar in gluten-ut. + "SPARK-34370: support Avro schema evolution (add column with avro.schema.url)", + "SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)" + ) + + testGluten("SPARK-34370: support Avro schema evolution (add column with avro.schema.url)") { + checkAvroSchemaEvolutionAddColumn( + avroSchemaUrlProperty("schemaWithOneField.avsc"), + avroSchemaUrlProperty("schemaWithTwoFields.avsc")) + } + + testGluten("SPARK-34370: support Avro schema evolution (remove column with avro.schema.url)") { + checkAvroSchemaEvolutionRemoveColumn( + avroSchemaUrlProperty("schemaWithTwoFields.avsc"), + avroSchemaUrlProperty("schemaWithOneField.avsc")) + } + + private def avroSchemaUrlProperty(fileName: String): String = { + val schemaPath = hiveResourcePath(fileName) + s"'avro.schema.url'='${schemaPath.toUri.toString}'" + } + + private def checkAvroSchemaEvolutionAddColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + checkAnswer( + spark.table("t"), + Row("col1_default", "col2_value", "1981-01-07") :: + Row("col1_value", "col2_value", "1983-04-27") :: Nil) + } + } + + private def checkAvroSchemaEvolutionRemoveColumn( + originalSerdeProperties: String, + evolvedSerdeProperties: String): Unit = { + withTable("t") { + sql(s""" + |CREATE TABLE t PARTITIONED BY (ds string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ($originalSerdeProperties) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |""".stripMargin) + sql("INSERT INTO t partition (ds='1983-04-27') VALUES ('col1_value', 'col2_value')") + sql(s"ALTER TABLE t SET SERDEPROPERTIES ($evolvedSerdeProperties)") + sql("INSERT INTO t partition (ds='1981-01-07') VALUES ('col2_value')") + checkAnswer( + spark.table("t"), + Row("col2_value", "1981-01-07") :: + Row("col2_value", "1983-04-27") :: Nil) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.scala new file mode 100644 index 000000000000..0f6b37016691 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveExplainSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveExplainSuite extends HiveExplainSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.scala new file mode 100644 index 000000000000..6cdf5f5845da --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHivePlanTest.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHivePlanTest extends HivePlanTest with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala new file mode 100644 index 000000000000..4b72a089864a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveQuerySuite.scala @@ -0,0 +1,73 @@ +/* + * 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.SparkFiles +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.HiveUtils.{builtinHiveVersion => hiveVersion} +import org.apache.spark.tags.SlowHiveTest + +import java.io.File + +@SlowHiveTest +class GlutenHiveQuerySuite + extends HiveQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + "ADD FILE command", + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + testGluten("ADD FILE command") { + val testFile = hiveResourcePath("data/files/v1.txt").toFile.toURI + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { + _ => Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + assert(sql("list files").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file").filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) + } + + testGluten("SPARK-33084: Add jar support Ivy URI in SQL") { + val testData = hiveResourcePath("data/files/sample.json").toUri + withTable("t") { + // Use transitive=false as it should be good enough to test the Ivy support in Hive ADD JAR. + sql( + s"ADD JAR ivy://org.apache.hive.hcatalog:hive-hcatalog-core:$hiveVersion?transitive=false") + sql("""CREATE TABLE t(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t""") + sql("SELECT * FROM src JOIN t on src.key = t.a") + assert( + sql("LIST JARS") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + assert( + sql("LIST JAR") + .filter(_.getString(0).contains( + s"org.apache.hive.hcatalog_hive-hcatalog-core-$hiveVersion.jar")) + .count() > 0) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.scala new file mode 100644 index 000000000000..d46c2eed7ad3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResolutionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveResolutionSuite + extends HiveResolutionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.scala new file mode 100644 index 000000000000..41a20e8ae6da --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveResourcePathSupport.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.hive.execution + +import org.apache.spark.SparkFunSuite + +import java.nio.file.Path + +/** Shared helpers for resolving workspace-backed Hive test resources. */ +trait GlutenHiveResourcePathSupport { + this: SparkFunSuite => + + protected lazy val hiveTestResourceDir: Path = + getWorkspaceFilePath("sql", "hive", "src", "test", "resources") + + final protected def hiveResourcePath(relativePath: String): Path = { + GlutenTestHiveTables.hiveResourcePath(hiveTestResourceDir, relativePath) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.scala new file mode 100644 index 000000000000..d537a9aae6f3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLViewSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSQLViewSuite extends HiveSQLViewSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.scala new file mode 100644 index 000000000000..158ad4f46e64 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveScriptTransformationSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveScriptTransformationSuite + extends HiveScriptTransformationSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.scala new file mode 100644 index 000000000000..92703dea3711 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeReadWriteSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeReadWriteSuite + extends HiveSerDeReadWriteSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.scala new file mode 100644 index 000000000000..3687714a592d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSerDeSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveSerDeSuite + extends HiveSerDeSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + /** + * Mostly copied from Spark's [[HiveSerDeSuite]] and [[GlutenTestSetWithSystemPropertyTrait]], and + * customized so `sales.txt` is loaded from the workspace-backed Hive test resources instead of + * via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + + TestHive.setCacheTables(false) + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"""LOAD DATA LOCAL INPATH '${hiveResourcePath("data/files/sales.txt").toFile.toURI}' + |INTO TABLE sales""".stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.scala new file mode 100644 index 000000000000..ac4903218fcf --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTableScanSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.sql.Row +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTableScanSuite + extends HiveTableScanSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed file because the upstream test resolves this resource + // from the spark-hive tests jar in gluten-ut. + "Spark-4077: timestamp query for null value" + ) + + testGluten("Spark-4077: timestamp query for null value") { + withTable("timestamp_query_null") { + sql(""" + |CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY ',' + |LINES TERMINATED BY '\n' + """.stripMargin) + val location = hiveResourcePath("data/files/issue-4077-data.txt").toFile.toURI + + sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert( + sql("SELECT time FROM timestamp_query_null LIMIT 2").collect() === + Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")), Row(null))) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.scala new file mode 100644 index 000000000000..d77f2c7d5780 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveTypeCoercionSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveTypeCoercionSuite + extends HiveTypeCoercionSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.scala new file mode 100644 index 000000000000..975fc9fb8997 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDAFSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDAFSuite extends HiveUDAFSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala new file mode 100644 index 000000000000..6a5ccdb625ed --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -0,0 +1,32 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHiveUDFSuite + extends HiveUDFSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..327b633b1f0e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenObjectHashAggregateSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenObjectHashAggregateSuite + extends ObjectHashAggregateSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.scala new file mode 100644 index 000000000000..14ffda794dcf --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruneHiveTablePartitionsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruneHiveTablePartitionsSuite + extends PruneHiveTablePartitionsSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala new file mode 100644 index 000000000000..aeb6e85b6a0c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenPruningSuite.scala @@ -0,0 +1,31 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenPruningSuite + extends PruningSuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveComparisonTestSupport { + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.scala new file mode 100644 index 000000000000..df750c09c49c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenSQLMetricsSuite extends SQLMetricsSuite with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..9af7bfca463c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenSQLQuerySuite.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.hive.execution + +import org.apache.spark.TestUtils +import org.apache.spark.sql.{GlutenTestSetWithSystemPropertyTrait, Row} +import org.apache.spark.tags.SlowHiveTest + +trait GlutenSQLQuerySuiteBase + extends SQLQuerySuiteBase + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + override def testNameBlackList: Seq[String] = Seq( + // Rewritten with a workspace-backed script path because getTestResourcePath resolves + // test-script.sh from the spark-hive tests jar in gluten-ut. + "script" + ) + + override def beforeAll(): Unit = { + super.beforeAll() + GlutenTestHiveTables.registerHiveQTestUtilsTables(hiveTestResourceDir) + } + + testGluten("script") { + withTempView("script_table") { + import spark.implicits._ + + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo")) + assume(TestUtils.testCommandAvailable("sed")) + val scriptFilePath = hiveResourcePath("test-script.sh").toFile.getCanonicalPath + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query = sql(s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + } +} + +@SlowHiveTest +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLQuerySuiteBase {} + +@SlowHiveTest +class GlutenSQLQuerySuiteAE extends SQLQuerySuiteAE with GlutenSQLQuerySuiteBase {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.scala new file mode 100644 index 000000000000..0eaa597a8680 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenTestHiveTables.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.hive.execution + +import org.apache.spark.sql.hive.test.TestHive.sparkSession +import org.apache.spark.sql.hive.test.TestHive.sparkSession.TestTable +import org.apache.spark.sql.hive.test.TestHiveQueryExecution + +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + +import java.nio.file.Path + +/** Wrappers around [[TestHive]] lazy-loaded table registrations. */ +object GlutenTestHiveTables { + def hiveResourcePath(resourceDir: Path, relativePath: String): Path = { + relativePath.split('/').foldLeft(resourceDir) { case (path, child) => path.resolve(child) } + } + + private def hiveDataFile(resourceDir: Path, fileName: String): String = { + hiveResourcePath(resourceDir, fileName).toAbsolutePath.normalize.toString + } + + implicit private class SqlCmd(sql: String) { + def cmd: () => Unit = { + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit + } + } + + def registerTestTable(testTable: TestTable): Unit = { + sparkSession.registerTestTable(testTable) + } + + def registerHiveQTestUtilsTables(resourceDir: Path): Unit = { + def createTableSQL(tblName: String): String = { + s"CREATE TABLE $tblName (key INT, value STRING) STORED AS textfile" + } + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable( + "src", + createTableSQL("src").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |INTO TABLE src""".stripMargin.cmd + ), + TestTable( + "src1", + createTableSQL("src1").cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv3.txt")}' + |INTO TABLE src1""".stripMargin.cmd + ), + TestTable( + "srcpart", + () => { + s"${createTableSQL("srcpart")} PARTITIONED BY (ds STRING, hr STRING)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "srcpart1", + () => { + s"${createTableSQL("srcpart1")} PARTITIONED BY (ds STRING, hr INT)".cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin.cmd.apply() + } + } + ), + TestTable( + "src_thrift", + () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() + + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + } + ), + TestTable( + "serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd + ), + TestTable( + "episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable( + "episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable( + "src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"""LOAD DATA LOCAL INPATH '${hiveDataFile(resourceDir, "data/files/json.txt")}' + |INTO TABLE src_json""".stripMargin.cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.scala new file mode 100644 index 000000000000..0d2798995580 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenUDAQuerySuite.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.hive.execution + +import org.apache.spark.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenHashUDAQuerySuite extends HashUDAQuerySuite with GlutenTestSetWithSystemPropertyTrait {} + +@SlowHiveTest +class GlutenHashUDAQueryWithControlledFallbackSuite + extends HashUDAQueryWithControlledFallbackSuite + with GlutenTestSetWithSystemPropertyTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala new file mode 100644 index 000000000000..701fbb8467a9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenWindowQuerySuite.scala @@ -0,0 +1,70 @@ +/* + * 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.sql.GlutenTestSetWithSystemPropertyTrait +import org.apache.spark.tags.SlowHiveTest + +@SlowHiveTest +class GlutenWindowQuerySuite + extends WindowQuerySuite + with GlutenTestSetWithSystemPropertyTrait + with GlutenHiveResourcePathSupport { + + /** + * Mostly copied from Spark's [[WindowQuerySuite]] and [[GlutenTestSetWithSystemPropertyTrait]], + * and customized so `part_tiny.txt` is loaded from the workspace-backed Hive test resources + * instead of via [[TestHive.getHiveFile]]. + */ + override def beforeAll(): Unit = { + System.setProperty("spark.plugins", "org.apache.gluten.GlutenPlugin") + System.setProperty("spark.memory.offHeap.enabled", "true") + System.setProperty("spark.memory.offHeap.size", "1024MB") + System.setProperty( + "spark.shuffle.manager", + "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + sql("DROP TABLE IF EXISTS part") + sql(""" + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) USING hive + """.stripMargin) + val testData1 = hiveResourcePath("data/files/part_tiny.txt").toFile.toURI + sql(s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + System.clearProperty("spark.plugins") + System.clearProperty("spark.memory.offHeap.enabled") + System.clearProperty("spark.memory.offHeap.size") + System.clearProperty("spark.shuffle.manager") + } + } +} diff --git a/pom.xml b/pom.xml index d083252c939c..2be98cafe128 100644 --- a/pom.xml +++ b/pom.xml @@ -1185,6 +1185,7 @@ 2.3.0 23 4.8 + 3.3.2 0.15.0 1.7.32 2.17.2 @@ -1210,6 +1211,7 @@ 2.4.0 24 4.9.3 + 3.3.4 0.15.0 2.0.6 2.19.0